diff --git a/.baseline/checkstyle/checkstyle.xml b/.baseline/checkstyle/checkstyle.xml index 570fb84d70a4..30dfabba8690 100644 --- a/.baseline/checkstyle/checkstyle.xml +++ b/.baseline/checkstyle/checkstyle.xml @@ -427,6 +427,11 @@ + + + + + diff --git a/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml b/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml index 9e7d6034fe2e..a04263e8c943 100644 --- a/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml +++ b/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml @@ -28,7 +28,8 @@ body: description: What Apache Iceberg version are you using? multiple: false options: - - "1.6.0 (latest release)" + - "1.6.1 (latest release)" + - "1.6.0" - "1.5.2" - "1.5.1" - "1.5.0" diff --git a/.github/workflows/delta-conversion-ci.yml b/.github/workflows/delta-conversion-ci.yml index cd16847cf95a..9326d9d533fd 100644 --- a/.github/workflows/delta-conversion-ci.yml +++ b/.github/workflows/delta-conversion-ci.yml @@ -35,6 +35,7 @@ on: - '.github/workflows/hive-ci.yml' - '.github/workflows/java-ci.yml' - '.github/workflows/jmh-benchmarks-ci.yml' + - '.github/workflows/kafka-connect-ci.yml' - '.github/workflows/labeler.yml' - '.github/workflows/licence-check.yml' - '.github/workflows/open-api.yml' @@ -51,6 +52,7 @@ on: - 'hive3-orc-bundle/**' - 'hive-runtime/**' - 'flink/**' + - 'kafka-connect/**' - 'pig/**' - 'docs/**' - 'site/**' @@ -88,7 +90,7 @@ jobs: key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle*', '**/gradle-wrapper.properties') }} restore-keys: ${{ runner.os }}-gradle- - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - - run: ./gradlew -DsparkVersions=3.5 -DscalaVersion=2.12 -DhiveVersions= -DflinkVersions= :iceberg-delta-lake:check -Pquick=true -x javadoc + - run: ./gradlew -DsparkVersions=3.5 -DscalaVersion=2.12 -DhiveVersions= -DkafkaVersions= -DflinkVersions= :iceberg-delta-lake:check -Pquick=true -x javadoc - uses: actions/upload-artifact@v4 if: failure() with: @@ -117,7 +119,7 @@ jobs: key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle*', '**/gradle-wrapper.properties') }} restore-keys: ${{ runner.os }}-gradle- - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - - run: ./gradlew -DsparkVersions=3.5 -DscalaVersion=2.13 -DhiveVersions= -DflinkVersions= :iceberg-delta-lake:check -Pquick=true -x javadoc + - run: ./gradlew -DsparkVersions=3.5 -DscalaVersion=2.13 -DhiveVersions= -DkafkaVersions= -DflinkVersions= :iceberg-delta-lake:check -Pquick=true -x javadoc - uses: actions/upload-artifact@v4 if: failure() with: diff --git a/.github/workflows/flink-ci.yml b/.github/workflows/flink-ci.yml index 370375783cc2..8ed555847861 100644 --- a/.github/workflows/flink-ci.yml +++ b/.github/workflows/flink-ci.yml @@ -35,6 +35,7 @@ on: - '.github/workflows/hive-ci.yml' - '.github/workflows/java-ci.yml' - '.github/workflows/jmh-benchmarks-ci.yml' + - '.github/workflows/kafka-connect-ci.yml' - '.github/workflows/labeler.yml' - '.github/workflows/licence-check.yml' - '.github/workflows/open-api.yml' @@ -50,6 +51,7 @@ on: - 'hive3/**' - 'hive3-orc-bundle/**' - 'hive-runtime/**' + - 'kafka-connect/**' - 'spark/**' - 'pig/**' - 'docs/**' @@ -91,7 +93,7 @@ jobs: key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle*', '**/gradle-wrapper.properties') }} restore-keys: ${{ runner.os }}-gradle- - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - - run: ./gradlew -DsparkVersions= -DhiveVersions= -DflinkVersions=${{ matrix.flink }} :iceberg-flink:iceberg-flink-${{ matrix.flink }}:check :iceberg-flink:iceberg-flink-runtime-${{ matrix.flink }}:check -Pquick=true -x javadoc + - run: ./gradlew -DsparkVersions= -DhiveVersions= -DkafkaVersions= -DflinkVersions=${{ matrix.flink }} :iceberg-flink:iceberg-flink-${{ matrix.flink }}:check :iceberg-flink:iceberg-flink-runtime-${{ matrix.flink }}:check -Pquick=true -x javadoc - uses: actions/upload-artifact@v4 if: failure() with: diff --git a/.github/workflows/hive-ci.yml b/.github/workflows/hive-ci.yml index 6ad9f58410d6..bcaf62cc07f8 100644 --- a/.github/workflows/hive-ci.yml +++ b/.github/workflows/hive-ci.yml @@ -35,6 +35,7 @@ on: - '.github/workflows/flink-ci.yml' - '.github/workflows/java-ci.yml' - '.github/workflows/jmh-benchmarks-ci.yml' + - '.github/workflows/kafka-connect-ci.yml' - '.github/workflows/labeler.yml' - '.github/workflows/licence-check.yml' - '.github/workflows/open-api.yml' @@ -49,6 +50,7 @@ on: - 'arrow/**' - 'spark/**' - 'flink/**' + - 'kafka-connect/**' - 'pig/**' - 'docs/**' - 'site/**' @@ -86,7 +88,7 @@ jobs: key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle*', '**/gradle-wrapper.properties') }} restore-keys: ${{ runner.os }}-gradle- - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - - run: ./gradlew -DsparkVersions= -DhiveVersions=2 -DflinkVersions= -Pquick=true :iceberg-mr:check :iceberg-hive-runtime:check -x javadoc + - run: ./gradlew -DsparkVersions= -DhiveVersions=2 -DflinkVersions= -DkafkaVersions= -Pquick=true :iceberg-mr:check :iceberg-hive-runtime:check -x javadoc - uses: actions/upload-artifact@v4 if: failure() with: @@ -115,7 +117,7 @@ jobs: key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle*', '**/gradle-wrapper.properties') }} restore-keys: ${{ runner.os }}-gradle- - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - - run: ./gradlew -DsparkVersions= -DhiveVersions=3 -DflinkVersions= -Pquick=true :iceberg-hive3-orc-bundle:check :iceberg-hive3:check :iceberg-hive-runtime:check -x javadoc + - run: ./gradlew -DsparkVersions= -DhiveVersions=3 -DflinkVersions= -DkafkaVersions= -Pquick=true :iceberg-hive3-orc-bundle:check :iceberg-hive3:check :iceberg-hive-runtime:check -x javadoc - uses: actions/upload-artifact@v4 if: failure() with: diff --git a/.github/workflows/java-ci.yml b/.github/workflows/java-ci.yml index 0d39ee8646ad..1da7a673a865 100644 --- a/.github/workflows/java-ci.yml +++ b/.github/workflows/java-ci.yml @@ -35,6 +35,7 @@ on: - '.github/workflows/flink-ci.yml' - '.github/workflows/hive-ci.yml' - '.github/workflows/jmh-benchmarks-ci.yml' + - '.github/workflows/kafka-connect-ci.yml' - '.github/workflows/labeler.yml' - '.github/workflows/licence-check.yml' - '.github/workflows/open-api.yml' @@ -82,7 +83,7 @@ jobs: key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle*', '**/gradle-wrapper.properties') }} restore-keys: ${{ runner.os }}-gradle- - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - - run: ./gradlew check -DsparkVersions= -DhiveVersions= -DflinkVersions= -Pquick=true -x javadoc + - run: ./gradlew check -DsparkVersions= -DhiveVersions= -DflinkVersions= -DkafkaVersions= -Pquick=true -x javadoc - uses: actions/upload-artifact@v4 if: failure() with: diff --git a/.github/workflows/kafka-connect-ci.yml b/.github/workflows/kafka-connect-ci.yml new file mode 100644 index 000000000000..98ec18a77953 --- /dev/null +++ b/.github/workflows/kafka-connect-ci.yml @@ -0,0 +1,105 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +name: "Kafka Connect CI" +on: + push: + branches: + - 'main' + - '0.*' + - '1.*' + - '2.*' + tags: + - 'apache-iceberg-**' + pull_request: + paths-ignore: + - '.github/ISSUE_TEMPLATE/**' + - '.github/workflows/api-binary-compatibility.yml' + - '.github/workflows/delta-conversion-ci.yml' + - '.github/workflows/flink-ci.yml' + - '.github/workflows/hive-ci.yml' + - '.github/workflows/java-ci.yml' + - '.github/workflows/jmh-benchmarks-ci.yml' + - '.github/workflows/labeler.yml' + - '.github/workflows/licence-check.yml' + - '.github/workflows/open-api.yml' + - '.github/workflows/publish-snapshot.yml' + - '.github/workflows/recurring-jmh-benchmarks.yml' + - '.github/workflows/site-ci.yml' + - '.github/workflows/spark-ci.yml' + - '.github/workflows/stale.yml' + - '.gitignore' + - '.asf.yml' + - 'dev/**' + - 'mr/**' + - 'flink/**' + - 'hive3/**' + - 'hive3-orc-bundle/**' + - 'hive-runtime/**' + - 'spark/**' + - 'pig/**' + - 'docs/**' + - 'site/**' + - 'open-api/**' + - 'format/**' + - '.gitattributes' + - 'README.md' + - 'CONTRIBUTING.md' + - 'LICENSE' + - 'NOTICE' + +concurrency: + group: ${{ github.workflow }}-${{ github.ref }} + cancel-in-progress: ${{ github.event_name == 'pull_request' }} + +jobs: + + kafka-connect-tests: + runs-on: ubuntu-22.04 + strategy: + matrix: + jvm: [11, 17, 21] + env: + SPARK_LOCAL_IP: localhost + steps: + - uses: actions/checkout@v4 + - uses: actions/setup-java@v4 + with: + distribution: zulu + java-version: ${{ matrix.jvm }} + - uses: actions/cache@v4 + with: + path: | + ~/.gradle/caches + ~/.gradle/wrapper + key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle*', '**/gradle-wrapper.properties') }} + restore-keys: ${{ runner.os }}-gradle- + - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts + - run: | + ./gradlew -DsparkVersions= -DhiveVersions= -DflinkVersions= -DkafkaVersions=3 \ + :iceberg-kafka-connect:iceberg-kafka-connect-events:check \ + :iceberg-kafka-connect:iceberg-kafka-connect:check \ + :iceberg-kafka-connect:iceberg-kafka-connect-runtime:check \ + -Pquick=true -x javadoc + - uses: actions/upload-artifact@v4 + if: failure() + with: + name: test logs + path: | + **/build/testlogs diff --git a/.github/workflows/publish-snapshot.yml b/.github/workflows/publish-snapshot.yml index 6acee54bde3e..7ff6b56da576 100644 --- a/.github/workflows/publish-snapshot.yml +++ b/.github/workflows/publish-snapshot.yml @@ -41,4 +41,4 @@ jobs: - run: | ./gradlew printVersion ./gradlew -DallModules publishApachePublicationToMavenRepository -PmavenUser=${{ secrets.NEXUS_USER }} -PmavenPassword=${{ secrets.NEXUS_PW }} - ./gradlew -DflinkVersions= -DsparkVersions=3.3,3.4,3.5 -DscalaVersion=2.13 -DhiveVersions= publishApachePublicationToMavenRepository -PmavenUser=${{ secrets.NEXUS_USER }} -PmavenPassword=${{ secrets.NEXUS_PW }} + ./gradlew -DflinkVersions= -DsparkVersions=3.3,3.4,3.5 -DscalaVersion=2.13 -DkafkaVersions=3 -DhiveVersions= publishApachePublicationToMavenRepository -PmavenUser=${{ secrets.NEXUS_USER }} -PmavenPassword=${{ secrets.NEXUS_PW }} diff --git a/.github/workflows/spark-ci.yml b/.github/workflows/spark-ci.yml index 1cc0425b73a3..b5d91d3cc76c 100644 --- a/.github/workflows/spark-ci.yml +++ b/.github/workflows/spark-ci.yml @@ -36,6 +36,7 @@ on: - '.github/workflows/hive-ci.yml' - '.github/workflows/java-ci.yml' - '.github/workflows/jmh-benchmarks-ci.yml' + - '.github/workflows/kafka-connect-ci.yml' - '.github/workflows/labeler.yml' - '.github/workflows/licence-check.yml' - '.github/workflows/open-api.yml' @@ -52,6 +53,7 @@ on: - 'hive3-orc-bundle/**' - 'hive-runtime/**' - 'flink/**' + - 'kafka-connect/**' - 'pig/**' - 'docs/**' - 'open-api/**' @@ -101,7 +103,7 @@ jobs: tool-cache: false - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - run: | - ./gradlew -DsparkVersions=${{ matrix.spark }} -DscalaVersion=${{ matrix.scala }} -DhiveVersions= -DflinkVersions= \ + ./gradlew -DsparkVersions=${{ matrix.spark }} -DscalaVersion=${{ matrix.scala }} -DhiveVersions= -DflinkVersions= -DkafkaVersions= \ :iceberg-spark:iceberg-spark-${{ matrix.spark }}_${{ matrix.scala }}:check \ :iceberg-spark:iceberg-spark-extensions-${{ matrix.spark }}_${{ matrix.scala }}:check \ :iceberg-spark:iceberg-spark-runtime-${{ matrix.spark }}_${{ matrix.scala }}:check \ diff --git a/.gitignore b/.gitignore index 6740a061e346..e4c9e1a16a27 100644 --- a/.gitignore +++ b/.gitignore @@ -37,7 +37,7 @@ spark/v3.4/spark/benchmark/* spark/v3.4/spark-extensions/benchmark/* spark/v3.5/spark/benchmark/* spark/v3.5/spark-extensions/benchmark/* -data/benchmark/* +*/benchmark/* __pycache__/ *.py[cod] @@ -66,3 +66,6 @@ metastore_db/ # Spark/metastore files spark-warehouse/ derby.log + +# jenv +.java-version diff --git a/.palantir/revapi.yml b/.palantir/revapi.yml index 9b8017f0beec..fade79326a49 100644 --- a/.palantir/revapi.yml +++ b/.palantir/revapi.yml @@ -873,6 +873,11 @@ acceptedBreaks: new: "method void org.apache.iceberg.encryption.Ciphers::()" justification: "Static utility class - should not have public constructor" "1.4.0": + org.apache.iceberg:iceberg-api: + - code: "java.class.defaultSerializationChanged" + old: "class org.apache.iceberg.types.Types.NestedField" + new: "class org.apache.iceberg.types.Types.NestedField" + justification: "Add default value APIs." org.apache.iceberg:iceberg-core: - code: "java.class.defaultSerializationChanged" old: "class org.apache.iceberg.PartitionData" @@ -1088,6 +1093,10 @@ acceptedBreaks: old: "class org.apache.iceberg.GenericManifestFile" new: "class org.apache.iceberg.GenericManifestFile" justification: "Serialization across versions is not supported" + - code: "java.class.defaultSerializationChanged" + old: "class org.apache.iceberg.io.WriteResult" + new: "class org.apache.iceberg.io.WriteResult" + justification: "Serialization across versions is not supported" - code: "java.class.removed" old: "enum org.apache.iceberg.BaseMetastoreTableOperations.CommitStatus" justification: "Removing deprecated code" diff --git a/LICENSE b/LICENSE index efb46dab44da..76f6113d9811 100644 --- a/LICENSE +++ b/LICENSE @@ -298,6 +298,7 @@ License: https://www.apache.org/licenses/LICENSE-2.0 This product includes code from Delta Lake. * AssignmentAlignmentSupport is an independent development but UpdateExpressionsSupport in Delta was used as a reference. +* RoaringPositionBitmap is a Java implementation of RoaringBitmapArray in Delta. Copyright: 2020 The Delta Lake Project Authors. Home page: https://delta.io/ diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/TestUtility.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/TestUtility.java index 072886f6b86b..430eb6a50b42 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/TestUtility.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/TestUtility.java @@ -65,7 +65,7 @@ public static AliyunOSSExtension initialize() { } else { LOG.info( "Initializing AliyunOSSExtension implementation with default AliyunOSSMockExtension"); - extension = AliyunOSSMockExtension.builder().silent().build(); + extension = AliyunOSSMockExtension.builder().build(); } return extension; diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSOutputStream.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSOutputStream.java index 8fc661e5be10..9a7b774b28bd 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSOutputStream.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSOutputStream.java @@ -70,7 +70,7 @@ public void testWrite() throws IOException { reset(ossMock); // Write large file. - writeAndVerify(ossMock, uri, randomData(32 * 1024 * 1024), arrayWrite); + writeAndVerify(ossMock, uri, randomData(32 * 1024), arrayWrite); verify(ossMock, times(1)).putObject(any()); reset(ossMock); } diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMock.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMock.java new file mode 100644 index 000000000000..7894c1857d55 --- /dev/null +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMock.java @@ -0,0 +1,569 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.aliyun.oss.mock; + +import com.aliyun.oss.OSSErrorCode; +import com.aliyun.oss.model.Bucket; +import com.sun.net.httpserver.HttpExchange; +import com.sun.net.httpserver.HttpHandler; +import com.sun.net.httpserver.HttpServer; +import java.io.FileInputStream; +import java.io.FilterInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.net.InetSocketAddress; +import java.time.ZoneId; +import java.time.ZonedDateTime; +import java.time.format.DateTimeFormatter; +import java.util.Collections; +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.io.ByteStreams; + +public class AliyunOSSMock { + + static final String PROP_ROOT_DIR = "root-dir"; + static final String ROOT_DIR_DEFAULT = "/tmp"; + + static final String PROP_HTTP_PORT = "server.port"; + static final int PORT_HTTP_PORT_DEFAULT = 9393; + + private final AliyunOSSMockLocalStore localStore; + private final HttpServer httpServer; + + public static AliyunOSSMock start(Map properties) throws IOException { + AliyunOSSMock mock = + new AliyunOSSMock( + properties.getOrDefault(PROP_ROOT_DIR, ROOT_DIR_DEFAULT).toString(), + Integer.parseInt( + properties.getOrDefault(PROP_HTTP_PORT, PORT_HTTP_PORT_DEFAULT).toString())); + mock.start(); + return mock; + } + + private AliyunOSSMock(String rootDir, int serverPort) throws IOException { + localStore = new AliyunOSSMockLocalStore(rootDir); + httpServer = HttpServer.create(new InetSocketAddress("localhost", serverPort), 0); + } + + private void start() { + httpServer.createContext("/", new AliyunHttpHandler()); + httpServer.start(); + } + + public void stop() { + httpServer.stop(0); + } + + private class AliyunHttpHandler implements HttpHandler { + + @Override + public void handle(HttpExchange httpExchange) throws IOException { + String request = httpExchange.getRequestURI().getPath().substring(1); + String[] requests = request.split("/"); + String bucketName = requests[0]; + if (requests.length == 1) { + // bucket operations + if (httpExchange.getRequestMethod().equals("PUT")) { + putBucket(bucketName, httpExchange); + } + if (httpExchange.getRequestMethod().equals("DELETE")) { + deleteBucket(bucketName, httpExchange); + } + } else { + // object operations + String objectName = requests[1]; + if (objectName.contains("?")) { + objectName = objectName.substring(0, objectName.indexOf("?")); + } + if (httpExchange.getRequestMethod().equals("PUT")) { + putObject(bucketName, objectName, httpExchange); + } + if (httpExchange.getRequestMethod().equals("DELETE")) { + deleteObject(bucketName, objectName, httpExchange); + } + if (httpExchange.getRequestMethod().equals("HEAD")) { + getObjectMeta(bucketName, objectName, httpExchange); + } + if (httpExchange.getRequestMethod().equals("GET")) { + getObject(bucketName, objectName, httpExchange); + } + } + } + + private void putBucket(String bucketName, HttpExchange httpExchange) throws IOException { + if (localStore.getBucket(bucketName) != null) { + String errorMessage = + createErrorResponse( + OSSErrorCode.BUCKET_ALREADY_EXISTS, bucketName + " already exists."); + handleResponse(httpExchange, 409, errorMessage, "application/xml"); + return; + } + localStore.createBucket(bucketName); + handleResponse(httpExchange, 200, "OK", "application/xml"); + } + + private void deleteBucket(String bucketName, HttpExchange httpExchange) throws IOException { + verifyBucketExistence(bucketName, httpExchange); + try { + localStore.deleteBucket(bucketName); + } catch (Exception e) { + String errorMessage = + createErrorResponse( + OSSErrorCode.BUCKET_NOT_EMPTY, "The bucket you tried to delete is not empty."); + handleResponse(httpExchange, 409, errorMessage, "application/xml"); + } + handleResponse(httpExchange, 200, "OK", "application/xml"); + } + + private void putObject(String bucketName, String objectName, HttpExchange httpExchange) + throws IOException { + verifyBucketExistence(bucketName, httpExchange); + + try (InputStream inputStream = httpExchange.getRequestBody()) { + ObjectMetadata metadata = + localStore.putObject( + bucketName, + objectName, + inputStream, + httpExchange.getRequestHeaders().getFirst("Content-Type"), + httpExchange.getRequestHeaders().getFirst("Content-Headers"), + ImmutableMap.of()); + + httpExchange.getResponseHeaders().add("ETag", metadata.getContentMD5()); + httpExchange + .getResponseHeaders() + .add("Last-Modified", createDate(metadata.getLastModificationDate())); + handleResponse(httpExchange, 200, "OK", "text/plain"); + } catch (Exception e) { + handleResponse(httpExchange, 500, "Internal Server Error", "text/plain"); + } + } + + private void deleteObject(String bucketName, String objectName, HttpExchange httpExchange) + throws IOException { + verifyBucketExistence(bucketName, httpExchange); + localStore.deleteObject(bucketName, objectName); + + handleResponse(httpExchange, 200, "OK", "text/plain"); + } + + private void getObjectMeta(String bucketName, String objectName, HttpExchange httpExchange) + throws IOException { + verifyBucketExistence(bucketName, httpExchange); + ObjectMetadata metadata = verifyObjectExistence(bucketName, objectName); + + if (metadata == null) { + String errorMessage = + createErrorResponse(OSSErrorCode.NO_SUCH_KEY, "The specify oss key does not exists."); + handleResponse(httpExchange, 404, errorMessage, "application/xml"); + } else { + httpExchange.getResponseHeaders().add("ETag", metadata.getContentMD5()); + httpExchange + .getResponseHeaders() + .add("Last-Modified", createDate(metadata.getLastModificationDate())); + httpExchange + .getResponseHeaders() + .add("Content-Length", Long.toString(metadata.getContentLength())); + + handleResponse(httpExchange, 200, "OK", "text/plain"); + } + } + + private void getObject(String bucketName, String objectName, HttpExchange httpExchange) + throws IOException { + verifyBucketExistence(bucketName, httpExchange); + + String filename = objectName; + ObjectMetadata metadata = verifyObjectExistence(bucketName, filename); + + if (metadata == null) { + String errorMessage = + createErrorResponse(OSSErrorCode.NO_SUCH_KEY, "The specify oss key does not exists."); + handleResponse(httpExchange, 404, errorMessage, "application/xml"); + return; + } + + Object range = httpExchange.getRequestHeaders().get("Range"); + if (range != null) { + range = range.toString().replace("[bytes=", "").replace("]", ""); + String[] ranges = range.toString().split("-"); + long rangeStart = -1; + if (!ranges[0].isEmpty()) { + rangeStart = Long.parseLong(ranges[0]); + } + long rangeEnd = -1; + if (ranges.length == 2 && !ranges[1].isEmpty()) { + rangeEnd = Long.parseLong(ranges[1]); + } + if (rangeEnd == -1) { + rangeEnd = Long.MAX_VALUE; + if (rangeStart == -1) { + rangeStart = 0; + } + } + + long fileSize = metadata.getContentLength(); + long bytesToRead = Math.min(fileSize - 1, rangeEnd) - rangeStart + 1; + long skipSize = rangeStart; + if (rangeStart == -1) { + bytesToRead = Math.min(fileSize - 1, rangeEnd); + skipSize = fileSize - rangeEnd; + } + if (rangeEnd == -1) { + bytesToRead = fileSize - rangeStart; + } + if (bytesToRead < 0 || fileSize < rangeStart) { + httpExchange.sendResponseHeaders(416, 1); + return; + } + + httpExchange.getResponseHeaders().add("Accept-Ranges", "bytes"); + httpExchange + .getResponseHeaders() + .add( + "Content-Range", + "bytes " + + rangeStart + + "-" + + (bytesToRead + rangeStart + 1) + + "/" + + metadata.getContentLength()); + httpExchange.getResponseHeaders().add("ETag", metadata.getContentMD5()); + httpExchange + .getResponseHeaders() + .add("Last-Modified", createDate(metadata.getLastModificationDate())); + httpExchange.getResponseHeaders().add("Content-Type", metadata.getContentType()); + httpExchange.getResponseHeaders().add("Content-Length", Long.toString(bytesToRead)); + httpExchange.sendResponseHeaders(206, bytesToRead); + try (OutputStream outputStream = httpExchange.getResponseBody()) { + try (FileInputStream fis = new FileInputStream(metadata.getDataFile())) { + fis.skip(skipSize); + ByteStreams.copy(new BoundedInputStream(fis, bytesToRead), outputStream); + } + } + } else { + httpExchange.getResponseHeaders().add("Accept-Ranges", "bytes"); + httpExchange.getResponseHeaders().add("ETag", metadata.getContentMD5()); + httpExchange + .getResponseHeaders() + .add("Last-Modified", createDate(metadata.getLastModificationDate())); + httpExchange.getResponseHeaders().add("Content-Type", metadata.getContentType()); + httpExchange.sendResponseHeaders(200, metadata.getContentLength()); + + try (OutputStream outputStream = httpExchange.getResponseBody()) { + try (FileInputStream fis = new FileInputStream(metadata.getDataFile())) { + ByteStreams.copy(fis, outputStream); + } + } + } + } + + private void verifyBucketExistence(String bucketName, HttpExchange httpExchange) + throws IOException { + Bucket bucket = localStore.getBucket(bucketName); + if (bucket == null) { + String errorMessage = + createErrorResponse( + OSSErrorCode.NO_SUCH_BUCKET, "The specified bucket does not exist."); + handleResponse(httpExchange, 404, errorMessage, "application/xml"); + } + } + + private ObjectMetadata verifyObjectExistence(String bucketName, String fileName) { + ObjectMetadata objectMetadata = null; + try { + objectMetadata = localStore.getObjectMetadata(bucketName, fileName); + } catch (IOException e) { + // no-op + } + + return objectMetadata; + } + + private void handleResponse( + HttpExchange httpExchange, int responseCode, String responsePayload, String contentType) + throws IOException { + OutputStream outputStream = httpExchange.getResponseBody(); + httpExchange.getResponseHeaders().put("Content-Type", Collections.singletonList(contentType)); + httpExchange.sendResponseHeaders(responseCode, responsePayload.length()); + outputStream.write(responsePayload.getBytes()); + outputStream.flush(); + outputStream.close(); + } + + private String createErrorResponse(String errorCode, String message) { + StringBuilder builder = new StringBuilder(); + builder.append(""); + builder.append("").append(errorCode).append(""); + builder.append("").append(message).append(""); + builder.append(""); + return builder.toString(); + } + + private String createDate(long timestamp) { + java.util.Date date = new java.util.Date(timestamp); + ZonedDateTime dateTime = date.toInstant().atZone(ZoneId.of("GMT")); + return dateTime.format(DateTimeFormatter.RFC_1123_DATE_TIME); + } + } + + /** + * Reads bytes up to a maximum length, if its count goes above that, it stops. + * + *

This is useful to wrap ServletInputStreams. The ServletInputStream will block if you try to + * read content from it that isn't there, because it doesn't know whether the content hasn't + * arrived yet or whether the content has finished. So, one of these, initialized with the + * Content-length sent in the ServletInputStream's header, will stop it blocking, providing it's + * been sent with a correct content length. + * + *

This code is borrowed from `org.apache.commons:commons-io` + */ + public class BoundedInputStream extends FilterInputStream { + + /** The max count of bytes to read. */ + private final long maxCount; + + /** The count of bytes read. */ + private long count; + + /** The marked position. */ + private long mark = -1; + + /** Flag if close should be propagated. */ + private boolean propagateClose = true; + + /** + * Constructs a new {@link BoundedInputStream} that wraps the given input stream and is + * unlimited. + * + * @param in The wrapped input stream. + */ + public BoundedInputStream(final InputStream in) { + this(in, -1); + } + + /** + * Constructs a new {@link BoundedInputStream} that wraps the given input stream and limits it + * to a certain size. + * + * @param inputStream The wrapped input stream. + * @param maxLength The maximum number of bytes to return. + */ + public BoundedInputStream(final InputStream inputStream, final long maxLength) { + // Some badly designed methods - e.g. the servlet API - overload length + // such that "-1" means stream finished + super(inputStream); + this.maxCount = maxLength; + } + + /** {@inheritDoc} */ + @Override + public int available() throws IOException { + if (isMaxLength()) { + onMaxLength(maxCount, count); + return 0; + } + return in.available(); + } + + /** + * Invokes the delegate's {@code close()} method if {@link #isPropagateClose()} is {@code true}. + * + * @throws IOException if an I/O error occurs. + */ + @Override + public void close() throws IOException { + if (propagateClose) { + in.close(); + } + } + + /** + * Gets the count of bytes read. + * + * @return The count of bytes read. + * @since 2.12.0 + */ + public long getCount() { + return count; + } + + /** + * Gets the max count of bytes to read. + * + * @return The max count of bytes to read. + * @since 2.12.0 + */ + public long getMaxLength() { + return maxCount; + } + + private boolean isMaxLength() { + return maxCount >= 0 && count >= maxCount; + } + + /** + * Tests whether the {@link #close()} method should propagate to the underling {@link + * InputStream}. + * + * @return {@code true} if calling {@link #close()} propagates to the {@code close()} method of + * the underlying stream or {@code false} if it does not. + */ + public boolean isPropagateClose() { + return propagateClose; + } + + /** + * Sets whether the {@link #close()} method should propagate to the underling {@link + * InputStream}. + * + * @param propagateClose {@code true} if calling {@link #close()} propagates to the {@code + * close()} method of the underlying stream or {@code false} if it does not. + */ + public void setPropagateClose(final boolean propagateClose) { + this.propagateClose = propagateClose; + } + + /** + * Invokes the delegate's {@code mark(int)} method. + * + * @param readlimit read ahead limit + */ + @Override + public synchronized void mark(final int readlimit) { + in.mark(readlimit); + mark = count; + } + + /** + * Invokes the delegate's {@code markSupported()} method. + * + * @return true if mark is supported, otherwise false + */ + @Override + public boolean markSupported() { + return in.markSupported(); + } + + /** + * A caller has caused a request that would cross the {@code maxLength} boundary. + * + * @param maxLength The max count of bytes to read. + * @param bytesRead The count of bytes read. + * @throws IOException Subclasses may throw. + * @since 2.12.0 + */ + protected void onMaxLength(final long maxLength, final long bytesRead) throws IOException { + // for subclasses + } + + /** + * Invokes the delegate's {@code read()} method if the current position is less than the limit. + * + * @return the byte read or -1 if the end of stream or the limit has been reached. + * @throws IOException if an I/O error occurs. + */ + @Override + public int read() throws IOException { + if (isMaxLength()) { + onMaxLength(maxCount, count); + return -1; + } + final int result = in.read(); + count++; + return result; + } + + /** + * Invokes the delegate's {@code read(byte[])} method. + * + * @param b the buffer to read the bytes into + * @return the number of bytes read or -1 if the end of stream or the limit has been reached. + * @throws IOException if an I/O error occurs. + */ + @Override + public int read(final byte[] b) throws IOException { + return this.read(b, 0, b.length); + } + + /** + * Invokes the delegate's {@code read(byte[], int, int)} method. + * + * @param b the buffer to read the bytes into + * @param off The start offset + * @param len The number of bytes to read + * @return the number of bytes read or -1 if the end of stream or the limit has been reached. + * @throws IOException if an I/O error occurs. + */ + @Override + public int read(final byte[] b, final int off, final int len) throws IOException { + if (isMaxLength()) { + onMaxLength(maxCount, count); + return -1; + } + final long maxRead = maxCount >= 0 ? Math.min(len, maxCount - count) : len; + final int bytesRead = in.read(b, off, (int) maxRead); + + if (bytesRead == -1) { + return -1; + } + + count += bytesRead; + return bytesRead; + } + + /** + * Invokes the delegate's {@code reset()} method. + * + * @throws IOException if an I/O error occurs. + */ + @Override + public synchronized void reset() throws IOException { + in.reset(); + count = mark; + } + + /** + * Invokes the delegate's {@code skip(long)} method. + * + * @param n the number of bytes to skip + * @return the actual number of bytes skipped + * @throws IOException if an I/O error occurs. + */ + @Override + public long skip(final long n) throws IOException { + final long toSkip = maxCount >= 0 ? Math.min(n, maxCount - count) : n; + final long skippedBytes = in.skip(toSkip); + count += skippedBytes; + return skippedBytes; + } + + /** + * Invokes the delegate's {@code toString()} method. + * + * @return the delegate's {@code toString()} + */ + @Override + public String toString() { + return in.toString(); + } + } +} diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockApp.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockApp.java deleted file mode 100644 index ea0ef0fe4de3..000000000000 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockApp.java +++ /dev/null @@ -1,158 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.aliyun.oss.mock; - -import java.util.List; -import java.util.Map; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.boot.Banner; -import org.springframework.boot.SpringApplication; -import org.springframework.boot.autoconfigure.EnableAutoConfiguration; -import org.springframework.boot.autoconfigure.security.servlet.SecurityAutoConfiguration; -import org.springframework.boot.builder.SpringApplicationBuilder; -import org.springframework.context.ConfigurableApplicationContext; -import org.springframework.context.annotation.Bean; -import org.springframework.context.annotation.ComponentScan; -import org.springframework.context.annotation.Configuration; -import org.springframework.core.convert.converter.Converter; -import org.springframework.http.MediaType; -import org.springframework.http.converter.xml.MappingJackson2XmlHttpMessageConverter; -import org.springframework.util.StringUtils; -import org.springframework.web.servlet.config.annotation.WebMvcConfigurer; - -@SuppressWarnings("checkstyle:AnnotationUseStyle") -@Configuration -@EnableAutoConfiguration( - exclude = {SecurityAutoConfiguration.class}, - excludeName = { - "org.springframework.boot.actuate.autoconfigure.security.servlet.ManagementWebSecurityAutoConfiguration" - }) -@ComponentScan -public class AliyunOSSMockApp { - - static final String PROP_ROOT_DIR = "root-dir"; - - static final String PROP_HTTP_PORT = "server.port"; - static final int PORT_HTTP_PORT_DEFAULT = 9393; - - static final String PROP_SILENT = "silent"; - - @Autowired private ConfigurableApplicationContext context; - - public static AliyunOSSMockApp start(Map properties, String... args) { - Map defaults = Maps.newHashMap(); - defaults.put(PROP_HTTP_PORT, PORT_HTTP_PORT_DEFAULT); - - Banner.Mode bannerMode = Banner.Mode.CONSOLE; - - if (Boolean.parseBoolean(String.valueOf(properties.remove(PROP_SILENT)))) { - defaults.put("logging.level.root", "WARN"); - bannerMode = Banner.Mode.OFF; - } - - final ConfigurableApplicationContext ctx = - new SpringApplicationBuilder(AliyunOSSMockApp.class) - .properties(defaults) - .properties(properties) - .bannerMode(bannerMode) - .run(args); - - return ctx.getBean(AliyunOSSMockApp.class); - } - - public void stop() { - SpringApplication.exit(context, () -> 0); - } - - @Configuration - static class Config implements WebMvcConfigurer { - - @Bean - Converter rangeConverter() { - return new RangeConverter(); - } - - /** - * Creates an HttpMessageConverter for XML. - * - * @return The configured {@link MappingJackson2XmlHttpMessageConverter}. - */ - @Bean - public MappingJackson2XmlHttpMessageConverter getMessageConverter() { - List mediaTypes = Lists.newArrayList(); - mediaTypes.add(MediaType.APPLICATION_XML); - mediaTypes.add(MediaType.APPLICATION_FORM_URLENCODED); - mediaTypes.add(MediaType.APPLICATION_OCTET_STREAM); - - final MappingJackson2XmlHttpMessageConverter xmlConverter = - new MappingJackson2XmlHttpMessageConverter(); - xmlConverter.setSupportedMediaTypes(mediaTypes); - - return xmlConverter; - } - } - - private static class RangeConverter implements Converter { - - private static final Pattern REQUESTED_RANGE_PATTERN = - Pattern.compile("^bytes=((\\d*)-(\\d*))((,\\d*-\\d*)*)"); - - @Override - public Range convert(String rangeString) { - Preconditions.checkNotNull(rangeString, "Range value should not be null."); - - final Range range; - - // parsing a range specification of format: "bytes=start-end" - multiple ranges not supported - final Matcher matcher = REQUESTED_RANGE_PATTERN.matcher(rangeString.trim()); - if (matcher.matches()) { - final String rangeStart = matcher.group(2); - final String rangeEnd = matcher.group(3); - - long start = StringUtils.isEmpty(rangeStart) ? -1L : Long.parseLong(rangeStart); - long end = StringUtils.isEmpty(rangeEnd) ? Long.MAX_VALUE : Long.parseLong(rangeEnd); - range = new Range(start, end); - - if (matcher.groupCount() == 5 && !"".equals(matcher.group(4))) { - throw new IllegalArgumentException( - "Unsupported range specification. Only single range specifications allowed"); - } - if (range.start() != -1 && range.start() < 0) { - throw new IllegalArgumentException( - "Unsupported range specification. A start byte must be supplied"); - } - - if (range.end() != -1 && range.end() < range.start()) { - throw new IllegalArgumentException( - "Range header is malformed. End byte is smaller than start byte."); - } - } else { - // Per Aliyun OSS behavior, return whole object content for illegal header - range = new Range(0, Long.MAX_VALUE); - } - - return range; - } - } -} diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockExtension.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockExtension.java index 9aae5b777a8d..d4cb10615094 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockExtension.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockExtension.java @@ -34,7 +34,7 @@ public class AliyunOSSMockExtension implements AliyunOSSExtension { private final Map properties; - private AliyunOSSMockApp ossMockApp; + private AliyunOSSMock ossMock; private AliyunOSSMockExtension(Map properties) { this.properties = properties; @@ -51,12 +51,16 @@ public String keyPrefix() { @Override public void start() { - ossMockApp = AliyunOSSMockApp.start(properties); + try { + ossMock = AliyunOSSMock.start(properties); + } catch (Exception e) { + throw new RuntimeException("Can't start OSS Mock"); + } } @Override public void stop() { - ossMockApp.stop(); + ossMock.stop(); } @Override @@ -65,12 +69,12 @@ public OSS createOSSClient() { String.format( "http://localhost:%s", properties.getOrDefault( - AliyunOSSMockApp.PROP_HTTP_PORT, AliyunOSSMockApp.PORT_HTTP_PORT_DEFAULT)); + AliyunOSSMock.PROP_HTTP_PORT, AliyunOSSMock.PORT_HTTP_PORT_DEFAULT)); return new OSSClientBuilder().build(endpoint, "foo", "bar"); } private File rootDir() { - Object rootDir = properties.get(AliyunOSSMockApp.PROP_ROOT_DIR); + Object rootDir = properties.get(AliyunOSSMock.PROP_ROOT_DIR); Preconditions.checkNotNull(rootDir, "Root directory cannot be null"); return new File(rootDir.toString()); } @@ -103,20 +107,15 @@ public void tearDownBucket(String bucket) { public static class Builder { private final Map props = Maps.newHashMap(); - public Builder silent() { - props.put(AliyunOSSMockApp.PROP_SILENT, true); - return this; - } - public AliyunOSSExtension build() { - String rootDir = (String) props.get(AliyunOSSMockApp.PROP_ROOT_DIR); + String rootDir = (String) props.get(AliyunOSSMock.PROP_ROOT_DIR); if (Strings.isNullOrEmpty(rootDir)) { File dir = new File( System.getProperty("java.io.tmpdir"), "oss-mock-file-store-" + System.currentTimeMillis()); rootDir = dir.getAbsolutePath(); - props.put(AliyunOSSMockApp.PROP_ROOT_DIR, rootDir); + props.put(AliyunOSSMock.PROP_ROOT_DIR, rootDir); } File root = new File(rootDir); root.deleteOnExit(); diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockLocalController.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockLocalController.java deleted file mode 100644 index 7f7546ec233b..000000000000 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockLocalController.java +++ /dev/null @@ -1,522 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.aliyun.oss.mock; - -import static org.springframework.http.HttpStatus.INTERNAL_SERVER_ERROR; -import static org.springframework.http.HttpStatus.OK; -import static org.springframework.http.HttpStatus.PARTIAL_CONTENT; -import static org.springframework.http.HttpStatus.REQUESTED_RANGE_NOT_SATISFIABLE; - -import com.aliyun.oss.OSSErrorCode; -import com.aliyun.oss.model.Bucket; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonRootName; -import java.io.FileInputStream; -import java.io.FilterInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import javax.servlet.ServletInputStream; -import javax.servlet.http.HttpServletRequest; -import javax.servlet.http.HttpServletResponse; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.io.ByteStreams; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.http.HttpHeaders; -import org.springframework.http.MediaType; -import org.springframework.http.ResponseEntity; -import org.springframework.web.bind.annotation.ControllerAdvice; -import org.springframework.web.bind.annotation.ExceptionHandler; -import org.springframework.web.bind.annotation.PathVariable; -import org.springframework.web.bind.annotation.RequestHeader; -import org.springframework.web.bind.annotation.RequestMapping; -import org.springframework.web.bind.annotation.RequestMethod; -import org.springframework.web.bind.annotation.RestController; -import org.springframework.web.servlet.mvc.method.annotation.ResponseEntityExceptionHandler; - -@RestController -public class AliyunOSSMockLocalController { - private static final Logger LOG = LoggerFactory.getLogger(AliyunOSSMockLocalController.class); - - @Autowired private AliyunOSSMockLocalStore localStore; - - private static String filenameFrom(@PathVariable String bucketName, HttpServletRequest request) { - String requestUri = request.getRequestURI(); - return requestUri.substring(requestUri.indexOf(bucketName) + bucketName.length() + 1); - } - - @RequestMapping(value = "/{bucketName}", method = RequestMethod.PUT, produces = "application/xml") - public void putBucket(@PathVariable String bucketName) throws IOException { - if (localStore.getBucket(bucketName) != null) { - throw new OssException( - 409, OSSErrorCode.BUCKET_ALREADY_EXISTS, bucketName + " already exists."); - } - - localStore.createBucket(bucketName); - } - - @RequestMapping( - value = "/{bucketName}", - method = RequestMethod.DELETE, - produces = "application/xml") - public void deleteBucket(@PathVariable String bucketName) throws IOException { - verifyBucketExistence(bucketName); - - localStore.deleteBucket(bucketName); - } - - @RequestMapping(value = "/{bucketName:.+}/**", method = RequestMethod.PUT) - public ResponseEntity putObject( - @PathVariable String bucketName, HttpServletRequest request) { - verifyBucketExistence(bucketName); - String filename = filenameFrom(bucketName, request); - try (ServletInputStream inputStream = request.getInputStream()) { - ObjectMetadata metadata = - localStore.putObject( - bucketName, - filename, - inputStream, - request.getContentType(), - request.getHeader(HttpHeaders.CONTENT_ENCODING), - ImmutableMap.of()); - - HttpHeaders responseHeaders = new HttpHeaders(); - responseHeaders.setETag("\"" + metadata.getContentMD5() + "\""); - responseHeaders.setLastModified(metadata.getLastModificationDate()); - - return new ResponseEntity<>(responseHeaders, OK); - } catch (Exception e) { - LOG.error("Failed to put object - bucket: {} - object: {}", bucketName, filename, e); - return new ResponseEntity<>(e.getMessage(), INTERNAL_SERVER_ERROR); - } - } - - @RequestMapping(value = "/{bucketName:.+}/**", method = RequestMethod.DELETE) - public void deleteObject(@PathVariable String bucketName, HttpServletRequest request) { - verifyBucketExistence(bucketName); - - localStore.deleteObject(bucketName, filenameFrom(bucketName, request)); - } - - @RequestMapping(value = "/{bucketName:.+}/**", method = RequestMethod.HEAD) - public ResponseEntity getObjectMeta( - @PathVariable String bucketName, HttpServletRequest request) { - verifyBucketExistence(bucketName); - ObjectMetadata metadata = verifyObjectExistence(bucketName, filenameFrom(bucketName, request)); - - HttpHeaders headers = new HttpHeaders(); - headers.setETag("\"" + metadata.getContentMD5() + "\""); - headers.setLastModified(metadata.getLastModificationDate()); - headers.setContentLength(metadata.getContentLength()); - - return new ResponseEntity<>(headers, OK); - } - - @SuppressWarnings("checkstyle:AnnotationUseStyle") - @RequestMapping( - value = "/{bucketName:.+}/**", - method = RequestMethod.GET, - produces = "application/xml") - public void getObject( - @PathVariable String bucketName, - @RequestHeader(value = "Range", required = false) Range range, - HttpServletRequest request, - HttpServletResponse response) - throws IOException { - verifyBucketExistence(bucketName); - - String filename = filenameFrom(bucketName, request); - ObjectMetadata metadata = verifyObjectExistence(bucketName, filename); - - if (range != null) { - long fileSize = metadata.getContentLength(); - long bytesToRead = Math.min(fileSize - 1, range.end()) - range.start() + 1; - long skipSize = range.start(); - if (range.start() == -1) { - bytesToRead = Math.min(fileSize - 1, range.end()); - skipSize = fileSize - range.end(); - } - if (range.end() == -1) { - bytesToRead = fileSize - range.start(); - } - if (bytesToRead < 0 || fileSize < range.start()) { - response.setStatus(REQUESTED_RANGE_NOT_SATISFIABLE.value()); - response.flushBuffer(); - return; - } - - response.setStatus(PARTIAL_CONTENT.value()); - response.setHeader(HttpHeaders.ACCEPT_RANGES, "bytes"); - response.setHeader( - HttpHeaders.CONTENT_RANGE, - String.format( - "bytes %s-%s/%s", - range.start(), bytesToRead + range.start() + 1, metadata.getContentLength())); - response.setHeader(HttpHeaders.ETAG, "\"" + metadata.getContentMD5() + "\""); - response.setDateHeader(HttpHeaders.LAST_MODIFIED, metadata.getLastModificationDate()); - response.setContentType(metadata.getContentType()); - response.setContentLengthLong(bytesToRead); - - try (OutputStream outputStream = response.getOutputStream()) { - try (FileInputStream fis = new FileInputStream(metadata.getDataFile())) { - fis.skip(skipSize); - ByteStreams.copy(new BoundedInputStream(fis, bytesToRead), outputStream); - } - } - } else { - response.setHeader(HttpHeaders.ACCEPT_RANGES, "bytes"); - response.setHeader(HttpHeaders.ETAG, "\"" + metadata.getContentMD5() + "\""); - response.setDateHeader(HttpHeaders.LAST_MODIFIED, metadata.getLastModificationDate()); - response.setContentType(metadata.getContentType()); - response.setContentLengthLong(metadata.getContentLength()); - - try (OutputStream outputStream = response.getOutputStream()) { - try (FileInputStream fis = new FileInputStream(metadata.getDataFile())) { - ByteStreams.copy(fis, outputStream); - } - } - } - } - - private void verifyBucketExistence(String bucketName) { - Bucket bucket = localStore.getBucket(bucketName); - if (bucket == null) { - throw new OssException( - 404, OSSErrorCode.NO_SUCH_BUCKET, "The specified bucket does not exist. "); - } - } - - private ObjectMetadata verifyObjectExistence(String bucketName, String filename) { - ObjectMetadata objectMetadata = null; - try { - objectMetadata = localStore.getObjectMetadata(bucketName, filename); - } catch (IOException e) { - LOG.error( - "Failed to get the object metadata, bucket: {}, object: {}.", bucketName, filename, e); - } - - if (objectMetadata == null) { - throw new OssException(404, OSSErrorCode.NO_SUCH_KEY, "The specify oss key does not exists."); - } - - return objectMetadata; - } - - @ControllerAdvice - public static class OSSMockExceptionHandler extends ResponseEntityExceptionHandler { - - @ExceptionHandler - public ResponseEntity handleOSSException(OssException ex) { - LOG.info("Responding with status {} - {}, {}", ex.status, ex.code, ex.message); - - ErrorResponse errorResponse = new ErrorResponse(); - errorResponse.setCode(ex.getCode()); - errorResponse.setMessage(ex.getMessage()); - - HttpHeaders headers = new HttpHeaders(); - headers.setContentType(MediaType.APPLICATION_XML); - - return ResponseEntity.status(ex.status).headers(headers).body(errorResponse); - } - } - - public static class OssException extends RuntimeException { - - private final int status; - private final String code; - private final String message; - - public OssException(final int status, final String code, final String message) { - super(message); - this.status = status; - this.code = code; - this.message = message; - } - - public String getCode() { - return code; - } - - @Override - public String getMessage() { - return message; - } - } - - @JsonRootName("Error") - public static class ErrorResponse { - @JsonProperty("Code") - private String code; - - @JsonProperty("Message") - private String message; - - public void setCode(String code) { - this.code = code; - } - - public void setMessage(String message) { - this.message = message; - } - } - - /** - * Reads bytes up to a maximum length, if its count goes above that, it stops. - * - *

This is useful to wrap ServletInputStreams. The ServletInputStream will block if you try to - * read content from it that isn't there, because it doesn't know whether the content hasn't - * arrived yet or whether the content has finished. So, one of these, initialized with the - * Content-length sent in the ServletInputStream's header, will stop it blocking, providing it's - * been sent with a correct content length. - * - *

This code is borrowed from `org.apache.commons:commons-io` - */ - public class BoundedInputStream extends FilterInputStream { - - /** The max count of bytes to read. */ - private final long maxCount; - - /** The count of bytes read. */ - private long count; - - /** The marked position. */ - private long mark = -1; - - /** Flag if close should be propagated. */ - private boolean propagateClose = true; - - /** - * Constructs a new {@link BoundedInputStream} that wraps the given input stream and is - * unlimited. - * - * @param in The wrapped input stream. - */ - public BoundedInputStream(final InputStream in) { - this(in, -1); - } - - /** - * Constructs a new {@link BoundedInputStream} that wraps the given input stream and limits it - * to a certain size. - * - * @param inputStream The wrapped input stream. - * @param maxLength The maximum number of bytes to return. - */ - public BoundedInputStream(final InputStream inputStream, final long maxLength) { - // Some badly designed methods - e.g. the servlet API - overload length - // such that "-1" means stream finished - super(inputStream); - this.maxCount = maxLength; - } - - /** {@inheritDoc} */ - @Override - public int available() throws IOException { - if (isMaxLength()) { - onMaxLength(maxCount, count); - return 0; - } - return in.available(); - } - - /** - * Invokes the delegate's {@code close()} method if {@link #isPropagateClose()} is {@code true}. - * - * @throws IOException if an I/O error occurs. - */ - @Override - public void close() throws IOException { - if (propagateClose) { - in.close(); - } - } - - /** - * Gets the count of bytes read. - * - * @return The count of bytes read. - * @since 2.12.0 - */ - public long getCount() { - return count; - } - - /** - * Gets the max count of bytes to read. - * - * @return The max count of bytes to read. - * @since 2.12.0 - */ - public long getMaxLength() { - return maxCount; - } - - private boolean isMaxLength() { - return maxCount >= 0 && count >= maxCount; - } - - /** - * Tests whether the {@link #close()} method should propagate to the underling {@link - * InputStream}. - * - * @return {@code true} if calling {@link #close()} propagates to the {@code close()} method of - * the underlying stream or {@code false} if it does not. - */ - public boolean isPropagateClose() { - return propagateClose; - } - - /** - * Sets whether the {@link #close()} method should propagate to the underling {@link - * InputStream}. - * - * @param propagateClose {@code true} if calling {@link #close()} propagates to the {@code - * close()} method of the underlying stream or {@code false} if it does not. - */ - public void setPropagateClose(final boolean propagateClose) { - this.propagateClose = propagateClose; - } - - /** - * Invokes the delegate's {@code mark(int)} method. - * - * @param readlimit read ahead limit - */ - @Override - public synchronized void mark(final int readlimit) { - in.mark(readlimit); - mark = count; - } - - /** - * Invokes the delegate's {@code markSupported()} method. - * - * @return true if mark is supported, otherwise false - */ - @Override - public boolean markSupported() { - return in.markSupported(); - } - - /** - * A caller has caused a request that would cross the {@code maxLength} boundary. - * - * @param maxLength The max count of bytes to read. - * @param bytesRead The count of bytes read. - * @throws IOException Subclasses may throw. - * @since 2.12.0 - */ - protected void onMaxLength(final long maxLength, final long bytesRead) throws IOException { - // for subclasses - } - - /** - * Invokes the delegate's {@code read()} method if the current position is less than the limit. - * - * @return the byte read or -1 if the end of stream or the limit has been reached. - * @throws IOException if an I/O error occurs. - */ - @Override - public int read() throws IOException { - if (isMaxLength()) { - onMaxLength(maxCount, count); - return -1; - } - final int result = in.read(); - count++; - return result; - } - - /** - * Invokes the delegate's {@code read(byte[])} method. - * - * @param b the buffer to read the bytes into - * @return the number of bytes read or -1 if the end of stream or the limit has been reached. - * @throws IOException if an I/O error occurs. - */ - @Override - public int read(final byte[] b) throws IOException { - return this.read(b, 0, b.length); - } - - /** - * Invokes the delegate's {@code read(byte[], int, int)} method. - * - * @param b the buffer to read the bytes into - * @param off The start offset - * @param len The number of bytes to read - * @return the number of bytes read or -1 if the end of stream or the limit has been reached. - * @throws IOException if an I/O error occurs. - */ - @Override - public int read(final byte[] b, final int off, final int len) throws IOException { - if (isMaxLength()) { - onMaxLength(maxCount, count); - return -1; - } - final long maxRead = maxCount >= 0 ? Math.min(len, maxCount - count) : len; - final int bytesRead = in.read(b, off, (int) maxRead); - - if (bytesRead == -1) { - return -1; - } - - count += bytesRead; - return bytesRead; - } - - /** - * Invokes the delegate's {@code reset()} method. - * - * @throws IOException if an I/O error occurs. - */ - @Override - public synchronized void reset() throws IOException { - in.reset(); - count = mark; - } - - /** - * Invokes the delegate's {@code skip(long)} method. - * - * @param n the number of bytes to skip - * @return the actual number of bytes skipped - * @throws IOException if an I/O error occurs. - */ - @Override - public long skip(final long n) throws IOException { - final long toSkip = maxCount >= 0 ? Math.min(n, maxCount - count) : n; - final long skippedBytes = in.skip(toSkip); - count += skippedBytes; - return skippedBytes; - } - - /** - * Invokes the delegate's {@code toString()} method. - * - * @return the delegate's {@code toString()} - */ - @Override - public String toString() { - return in.toString(); - } - } -} diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockLocalStore.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockLocalStore.java index f7a4b72e4b97..521b87e31e80 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockLocalStore.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockLocalStore.java @@ -46,11 +46,7 @@ import org.apache.iceberg.relocated.com.google.common.io.ByteStreams; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.springframework.beans.factory.annotation.Value; -import org.springframework.http.MediaType; -import org.springframework.stereotype.Component; -@Component public class AliyunOSSMockLocalStore { private static final Logger LOG = LoggerFactory.getLogger(AliyunOSSMockLocalStore.class); @@ -61,8 +57,7 @@ public class AliyunOSSMockLocalStore { private final ObjectMapper objectMapper = new ObjectMapper(); - public AliyunOSSMockLocalStore( - @Value("${" + AliyunOSSMockApp.PROP_ROOT_DIR + ":}") String rootDir) { + public AliyunOSSMockLocalStore(String rootDir) { Preconditions.checkNotNull(rootDir, "Root directory cannot be null"); this.root = new File(rootDir); @@ -121,8 +116,7 @@ void deleteBucket(String bucketName) throws IOException { File dir = new File(root, bucket.getName()); if (Files.walk(dir.toPath()).anyMatch(p -> p.toFile().isFile())) { - throw new AliyunOSSMockLocalController.OssException( - 409, OSSErrorCode.BUCKET_NOT_EMPTY, "The bucket you tried to delete is not empty. "); + throw new RuntimeException(OSSErrorCode.BUCKET_NOT_EMPTY); } try (Stream walk = Files.walk(dir.toPath())) { @@ -156,7 +150,9 @@ ObjectMetadata putObject( metadata.setContentLength(dataFile.length()); metadata.setContentMD5(md5sum(dataFile.getAbsolutePath())); metadata.setContentType( - contentType != null ? contentType : MediaType.APPLICATION_OCTET_STREAM_VALUE); + contentType != null + ? contentType + : "application/octet"); // MediaType.APPLICATION_OCTET_STREAM_VALUE metadata.setContentEncoding(contentEncoding); metadata.setDataFile(dataFile.getAbsolutePath()); metadata.setMetaFile(metaFile.getAbsolutePath()); diff --git a/api/src/main/java/org/apache/iceberg/ContentFile.java b/api/src/main/java/org/apache/iceberg/ContentFile.java index 4bb8a78289de..a4d97fa93fa1 100644 --- a/api/src/main/java/org/apache/iceberg/ContentFile.java +++ b/api/src/main/java/org/apache/iceberg/ContentFile.java @@ -29,6 +29,14 @@ * @param the concrete Java class of a ContentFile instance. */ public interface ContentFile { + /** + * Returns the path of the manifest which this file is referenced in or null if it was not read + * from a manifest. + */ + default String manifestLocation() { + return null; + } + /** * Returns the ordinal position of the file in a manifest, or null if it was not read from a * manifest. @@ -43,9 +51,19 @@ public interface ContentFile { */ FileContent content(); - /** Returns fully qualified path to the file, suitable for constructing a Hadoop Path. */ + /** + * Returns fully qualified path to the file, suitable for constructing a Hadoop Path. + * + * @deprecated since 1.7.0, will be removed in 2.0.0; use {@link #location()} instead. + */ + @Deprecated CharSequence path(); + /** Return the fully qualified path to the file. */ + default String location() { + return path().toString(); + } + /** Returns format of the file. */ FileFormat format(); diff --git a/api/src/main/java/org/apache/iceberg/DataFile.java b/api/src/main/java/org/apache/iceberg/DataFile.java index 59b329c500c7..02ad0aff3128 100644 --- a/api/src/main/java/org/apache/iceberg/DataFile.java +++ b/api/src/main/java/org/apache/iceberg/DataFile.java @@ -102,6 +102,7 @@ public interface DataFile extends ContentFile { int PARTITION_ID = 102; String PARTITION_NAME = "partition"; String PARTITION_DOC = "Partition data tuple, schema based on the partition spec"; + // NEXT ID TO ASSIGN: 142 static StructType getType(StructType partitionType) { @@ -126,7 +127,9 @@ static StructType getType(StructType partitionType) { SORT_ORDER_ID); } - /** @return the content stored in the file; one of DATA, POSITION_DELETES, or EQUALITY_DELETES */ + /** + * @return the content stored in the file; one of DATA, POSITION_DELETES, or EQUALITY_DELETES + */ @Override default FileContent content() { return FileContent.DATA; diff --git a/api/src/main/java/org/apache/iceberg/PartitionSpec.java b/api/src/main/java/org/apache/iceberg/PartitionSpec.java index 08a1c4f9ecfd..9b74893f1831 100644 --- a/api/src/main/java/org/apache/iceberg/PartitionSpec.java +++ b/api/src/main/java/org/apache/iceberg/PartitionSpec.java @@ -371,6 +371,7 @@ public static class Builder { new AtomicInteger(unpartitionedLastAssignedId()); // check if there are conflicts between partition and schema field name private boolean checkConflicts = true; + private boolean caseSensitive = true; private Builder(Schema schema) { this.schema = schema; @@ -390,7 +391,8 @@ Builder checkConflicts(boolean check) { } private void checkAndAddPartitionName(String name, Integer sourceColumnId) { - Types.NestedField schemaField = schema.findField(name); + Types.NestedField schemaField = + this.caseSensitive ? schema.findField(name) : schema.caseInsensitiveFindField(name); if (checkConflicts) { if (sourceColumnId != null) { // for identity transform case we allow conflicts between partition and schema field name @@ -427,20 +429,31 @@ private void checkForRedundantPartitions(PartitionField field) { dedupFields.put(dedupKey, field); } + public Builder caseSensitive(boolean sensitive) { + this.caseSensitive = sensitive; + return this; + } + public Builder withSpecId(int newSpecId) { this.specId = newSpecId; return this; } private Types.NestedField findSourceColumn(String sourceName) { - Types.NestedField sourceColumn = schema.findField(sourceName); + Types.NestedField sourceColumn = + this.caseSensitive + ? schema.findField(sourceName) + : schema.caseInsensitiveFindField(sourceName); Preconditions.checkArgument( sourceColumn != null, "Cannot find source column: %s", sourceName); return sourceColumn; } Builder identity(String sourceName, String targetName) { - Types.NestedField sourceColumn = findSourceColumn(sourceName); + return identity(findSourceColumn(sourceName), targetName); + } + + private Builder identity(Types.NestedField sourceColumn, String targetName) { checkAndAddPartitionName(targetName, sourceColumn.fieldId()); PartitionField field = new PartitionField( @@ -451,12 +464,16 @@ Builder identity(String sourceName, String targetName) { } public Builder identity(String sourceName) { - return identity(sourceName, sourceName); + Types.NestedField sourceColumn = findSourceColumn(sourceName); + return identity(sourceColumn, schema.findColumnName(sourceColumn.fieldId())); } public Builder year(String sourceName, String targetName) { + return year(findSourceColumn(sourceName), targetName); + } + + private Builder year(Types.NestedField sourceColumn, String targetName) { checkAndAddPartitionName(targetName); - Types.NestedField sourceColumn = findSourceColumn(sourceName); PartitionField field = new PartitionField(sourceColumn.fieldId(), nextFieldId(), targetName, Transforms.year()); checkForRedundantPartitions(field); @@ -465,12 +482,17 @@ public Builder year(String sourceName, String targetName) { } public Builder year(String sourceName) { - return year(sourceName, sourceName + "_year"); + Types.NestedField sourceColumn = findSourceColumn(sourceName); + String columnName = schema.findColumnName(sourceColumn.fieldId()); + return year(sourceColumn, columnName + "_year"); } public Builder month(String sourceName, String targetName) { + return month(findSourceColumn(sourceName), targetName); + } + + private Builder month(Types.NestedField sourceColumn, String targetName) { checkAndAddPartitionName(targetName); - Types.NestedField sourceColumn = findSourceColumn(sourceName); PartitionField field = new PartitionField(sourceColumn.fieldId(), nextFieldId(), targetName, Transforms.month()); checkForRedundantPartitions(field); @@ -479,12 +501,17 @@ public Builder month(String sourceName, String targetName) { } public Builder month(String sourceName) { - return month(sourceName, sourceName + "_month"); + Types.NestedField sourceColumn = findSourceColumn(sourceName); + String columnName = schema.findColumnName(sourceColumn.fieldId()); + return month(sourceColumn, columnName + "_month"); } public Builder day(String sourceName, String targetName) { + return day(findSourceColumn(sourceName), targetName); + } + + private Builder day(Types.NestedField sourceColumn, String targetName) { checkAndAddPartitionName(targetName); - Types.NestedField sourceColumn = findSourceColumn(sourceName); PartitionField field = new PartitionField(sourceColumn.fieldId(), nextFieldId(), targetName, Transforms.day()); checkForRedundantPartitions(field); @@ -493,12 +520,17 @@ public Builder day(String sourceName, String targetName) { } public Builder day(String sourceName) { - return day(sourceName, sourceName + "_day"); + Types.NestedField sourceColumn = findSourceColumn(sourceName); + String columnName = schema.findColumnName(sourceColumn.fieldId()); + return day(sourceColumn, columnName + "_day"); } public Builder hour(String sourceName, String targetName) { + return hour(findSourceColumn(sourceName), targetName); + } + + private Builder hour(Types.NestedField sourceColumn, String targetName) { checkAndAddPartitionName(targetName); - Types.NestedField sourceColumn = findSourceColumn(sourceName); PartitionField field = new PartitionField(sourceColumn.fieldId(), nextFieldId(), targetName, Transforms.hour()); checkForRedundantPartitions(field); @@ -507,12 +539,17 @@ public Builder hour(String sourceName, String targetName) { } public Builder hour(String sourceName) { - return hour(sourceName, sourceName + "_hour"); + Types.NestedField sourceColumn = findSourceColumn(sourceName); + String columnName = schema.findColumnName(sourceColumn.fieldId()); + return hour(sourceColumn, columnName + "_hour"); } public Builder bucket(String sourceName, int numBuckets, String targetName) { + return bucket(findSourceColumn(sourceName), numBuckets, targetName); + } + + private Builder bucket(Types.NestedField sourceColumn, int numBuckets, String targetName) { checkAndAddPartitionName(targetName); - Types.NestedField sourceColumn = findSourceColumn(sourceName); fields.add( new PartitionField( sourceColumn.fieldId(), nextFieldId(), targetName, Transforms.bucket(numBuckets))); @@ -520,12 +557,17 @@ public Builder bucket(String sourceName, int numBuckets, String targetName) { } public Builder bucket(String sourceName, int numBuckets) { - return bucket(sourceName, numBuckets, sourceName + "_bucket"); + Types.NestedField sourceColumn = findSourceColumn(sourceName); + String columnName = schema.findColumnName(sourceColumn.fieldId()); + return bucket(sourceColumn, numBuckets, columnName + "_bucket"); } public Builder truncate(String sourceName, int width, String targetName) { + return truncate(findSourceColumn(sourceName), width, targetName); + } + + private Builder truncate(Types.NestedField sourceColumn, int width, String targetName) { checkAndAddPartitionName(targetName); - Types.NestedField sourceColumn = findSourceColumn(sourceName); fields.add( new PartitionField( sourceColumn.fieldId(), nextFieldId(), targetName, Transforms.truncate(width))); @@ -533,11 +575,16 @@ public Builder truncate(String sourceName, int width, String targetName) { } public Builder truncate(String sourceName, int width) { - return truncate(sourceName, width, sourceName + "_trunc"); + Types.NestedField sourceColumn = findSourceColumn(sourceName); + String columnName = schema.findColumnName(sourceColumn.fieldId()); + return truncate(sourceColumn, width, columnName + "_trunc"); } public Builder alwaysNull(String sourceName, String targetName) { - Types.NestedField sourceColumn = findSourceColumn(sourceName); + return alwaysNull(findSourceColumn(sourceName), targetName); + } + + private Builder alwaysNull(Types.NestedField sourceColumn, String targetName) { checkAndAddPartitionName( targetName, sourceColumn.fieldId()); // can duplicate a source column name fields.add( @@ -547,7 +594,9 @@ public Builder alwaysNull(String sourceName, String targetName) { } public Builder alwaysNull(String sourceName) { - return alwaysNull(sourceName, sourceName + "_null"); + Types.NestedField sourceColumn = findSourceColumn(sourceName); + String columnName = schema.findColumnName(sourceColumn.fieldId()); + return alwaysNull(sourceColumn, columnName + "_null"); } // add a partition field with an auto-increment partition field id starting from diff --git a/api/src/main/java/org/apache/iceberg/RowDelta.java b/api/src/main/java/org/apache/iceberg/RowDelta.java index 624f6c15d20b..a5e3fa477ba9 100644 --- a/api/src/main/java/org/apache/iceberg/RowDelta.java +++ b/api/src/main/java/org/apache/iceberg/RowDelta.java @@ -46,6 +46,17 @@ public interface RowDelta extends SnapshotUpdate { */ RowDelta addDeletes(DeleteFile deletes); + /** + * Removes a rewritten {@link DeleteFile} from the table. + * + * @param deletes a delete file that can be removed from the table + * @return this for method chaining + */ + default RowDelta removeDeletes(DeleteFile deletes) { + throw new UnsupportedOperationException( + getClass().getName() + " does not implement removeDeletes"); + } + /** * Set the snapshot ID used in any reads for this operation. * diff --git a/api/src/main/java/org/apache/iceberg/Schema.java b/api/src/main/java/org/apache/iceberg/Schema.java index 7ff712b62790..9bcf691f5a03 100644 --- a/api/src/main/java/org/apache/iceberg/Schema.java +++ b/api/src/main/java/org/apache/iceberg/Schema.java @@ -54,6 +54,8 @@ public class Schema implements Serializable { private static final Joiner NEWLINE = Joiner.on('\n'); private static final String ALL_COLUMNS = "*"; private static final int DEFAULT_SCHEMA_ID = 0; + private static final Map MIN_FORMAT_VERSIONS = + ImmutableMap.of(Type.TypeID.TIMESTAMP_NANO, 3); private final StructType struct; private final int schemaId; @@ -573,4 +575,27 @@ private List reassignIds(List columns, TypeUtil.GetID }); return res.asStructType().fields(); } + + /** + * Check the compatibility of the schema with a format version. + * + *

This validates that the schema does not contain types that were released in later format + * versions. + * + * @param schema a Schema + * @param formatVersion table format version + */ + public static void checkCompatibility(Schema schema, int formatVersion) { + // check the type in each field + for (NestedField field : schema.lazyIdToField().values()) { + Integer minFormatVersion = MIN_FORMAT_VERSIONS.get(field.type().typeId()); + Preconditions.checkState( + minFormatVersion == null || formatVersion >= minFormatVersion, + "Invalid type in v%s schema: %s %s is not supported until v%s", + formatVersion, + schema.findColumnName(field.fieldId()), + field.type(), + minFormatVersion); + } + } } diff --git a/api/src/main/java/org/apache/iceberg/UpdatePartitionSpec.java b/api/src/main/java/org/apache/iceberg/UpdatePartitionSpec.java index eeb596d42d5c..a4994d22001d 100644 --- a/api/src/main/java/org/apache/iceberg/UpdatePartitionSpec.java +++ b/api/src/main/java/org/apache/iceberg/UpdatePartitionSpec.java @@ -132,5 +132,5 @@ public interface UpdatePartitionSpec extends PendingUpdate { default UpdatePartitionSpec addNonDefaultSpec() { throw new UnsupportedOperationException( this.getClass().getName() + " doesn't implement addNonDefaultSpec()"); - }; + } } diff --git a/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java b/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java index 2d6ff2679a17..61750d83fc79 100644 --- a/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java +++ b/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java @@ -70,4 +70,22 @@ default RewritePositionDeleteFiles rewritePositionDeletes(Table table) { throw new UnsupportedOperationException( this.getClass().getName() + " does not implement rewritePositionDeletes"); } + + /** Instantiates an action to compute table stats. */ + default ComputeTableStats computeTableStats(Table table) { + throw new UnsupportedOperationException( + this.getClass().getName() + " does not implement computeTableStats"); + } + + /** Instantiates an action to rewrite all absolute paths in table metadata. */ + default RewriteTablePath rewriteTablePath(Table table) { + throw new UnsupportedOperationException( + this.getClass().getName() + " does not implement rewriteTablePath"); + } + + /** Instantiates an action to remove dangling delete files from current snapshot. */ + default RemoveDanglingDeleteFiles removeDanglingDeleteFiles(Table table) { + throw new UnsupportedOperationException( + this.getClass().getName() + " does not implement removeDanglingDeleteFiles"); + } } diff --git a/api/src/main/java/org/apache/iceberg/actions/ComputeTableStats.java b/api/src/main/java/org/apache/iceberg/actions/ComputeTableStats.java new file mode 100644 index 000000000000..04449d591657 --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/actions/ComputeTableStats.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.actions; + +import org.apache.iceberg.StatisticsFile; + +/** An action that collects statistics of an Iceberg table and writes to Puffin files. */ +public interface ComputeTableStats extends Action { + /** + * Choose the set of columns to collect stats, by default all columns are chosen. + * + * @param columns a set of column names to be analyzed + * @return this for method chaining + */ + ComputeTableStats columns(String... columns); + + /** + * Choose the table snapshot to compute stats, by default the current snapshot is used. + * + * @param snapshotId long ID of the snapshot for which stats need to be computed + * @return this for method chaining + */ + ComputeTableStats snapshot(long snapshotId); + + /** The result of table statistics collection. */ + interface Result { + + /** Returns statistics file or none if no statistics were collected. */ + StatisticsFile statisticsFile(); + } +} diff --git a/api/src/main/java/org/apache/iceberg/actions/RemoveDanglingDeleteFiles.java b/api/src/main/java/org/apache/iceberg/actions/RemoveDanglingDeleteFiles.java new file mode 100644 index 000000000000..b0ef0d5e35f8 --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/actions/RemoveDanglingDeleteFiles.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.actions; + +import org.apache.iceberg.DeleteFile; + +/** + * An action that removes dangling delete files from the current snapshot. A delete file is dangling + * if its deletes no longer applies to any live data files. + */ +public interface RemoveDanglingDeleteFiles + extends Action { + + /** An action that remove dangling deletes. */ + interface Result { + /** Return removed deletes. */ + Iterable removedDeleteFiles(); + } +} diff --git a/api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java b/api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java index f6ef40270852..589b9017741e 100644 --- a/api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java +++ b/api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java @@ -106,6 +106,18 @@ public interface RewriteDataFiles boolean USE_STARTING_SEQUENCE_NUMBER_DEFAULT = true; + /** + * Remove dangling delete files from the current snapshot after compaction. A delete file is + * considered dangling if it does not apply to any live data files. + * + *

Both equality and position dangling delete files will be removed. + * + *

Defaults to false. + */ + String REMOVE_DANGLING_DELETES = "remove-dangling-deletes"; + + boolean REMOVE_DANGLING_DELETES_DEFAULT = false; + /** * Forces the rewrite job order based on the value. * @@ -216,6 +228,10 @@ default long rewrittenBytesCount() { default int failedDataFilesCount() { return rewriteFailures().stream().mapToInt(FileGroupFailureResult::dataFilesCount).sum(); } + + default int removedDeleteFilesCount() { + return 0; + } } /** diff --git a/api/src/main/java/org/apache/iceberg/actions/RewriteTablePath.java b/api/src/main/java/org/apache/iceberg/actions/RewriteTablePath.java new file mode 100644 index 000000000000..b7aed67396a5 --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/actions/RewriteTablePath.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.actions; + +/** + * An action that rewrites the table's metadata files to a staging directory, replacing all source + * prefixes in absolute paths with a specified target prefix. There are two modes: + * + *

    + *
  • Complete copy: Rewrites all metadata files to the staging directory. + *
  • Incremental copy: Rewrites a subset of metadata files to the staging directory, + * consisting of metadata files added since a specified start version and/or until end + * version. The start/end version is identified by the name of a metadata.json file, and all + * metadata files added before/after these file are marked for rewrite. + *
+ * + * This action can be used as the starting point to fully or incrementally copy an Iceberg table + * located under the source prefix to the target prefix. + * + *

The action returns the following: + * + *

    + *
  1. The name of the latest metadata.json rewritten to staging location. After the files are + * copied, this will be the root of the copied table. + *
  2. A list of all files added to the table between startVersion and endVersion, including their + * original and target paths under the target prefix. This list covers both original and + * rewritten files, allowing for copying to the target paths to form the copied table. + *
+ */ +public interface RewriteTablePath extends Action { + + /** + * Configure a source prefix that will be replaced by the specified target prefix in all paths + * + * @param sourcePrefix the source prefix to be replaced + * @param targetPrefix the target prefix + * @return this for method chaining + */ + RewriteTablePath rewriteLocationPrefix(String sourcePrefix, String targetPrefix); + + /** + * First metadata version to rewrite, identified by name of a metadata.json file in the table's + * metadata log. It is optional, if provided then this action will only rewrite metadata files + * added after this version. + * + * @param startVersion name of a metadata.json file. For example, + * "00001-8893aa9e-f92e-4443-80e7-cfa42238a654.metadata.json". + * @return this for method chaining + */ + RewriteTablePath startVersion(String startVersion); + + /** + * Last metadata version to rewrite, identified by name of a metadata.json file in the table's + * metadata log. It is optional, if provided then this action will only rewrite metadata files + * added before this file, including the file itself. + * + * @param endVersion name of a metadata.json file. For example, + * "00001-8893aa9e-f92e-4443-80e7-cfa42238a654.metadata.json". + * @return this for method chaining + */ + RewriteTablePath endVersion(String endVersion); + + /** + * Custom staging location. It is optional. By default, staging location is a subdirectory under + * table's metadata directory. + * + * @param stagingLocation the staging location + * @return this for method chaining + */ + RewriteTablePath stagingLocation(String stagingLocation); + + /** The action result that contains a summary of the execution. */ + interface Result { + /** Staging location of rewritten files */ + String stagingLocation(); + + /** + * Path to a comma-separated list of source and target paths for all files added to the table + * between startVersion and endVersion, including original data files and metadata files + * rewritten to staging. + */ + String fileListLocation(); + + /** Name of latest metadata file version */ + String latestVersion(); + } +} diff --git a/api/src/main/java/org/apache/iceberg/encryption/KmsClient.java b/api/src/main/java/org/apache/iceberg/encryption/KmsClient.java index 3ebda7be27f2..87dd2b286b12 100644 --- a/api/src/main/java/org/apache/iceberg/encryption/KmsClient.java +++ b/api/src/main/java/org/apache/iceberg/encryption/KmsClient.java @@ -23,7 +23,9 @@ import java.util.Map; /** A minimum client interface to connect to a key management service (KMS). */ -/** @deprecated the API will be removed in v2.0.0 (replaced with KeyManagementClient interface). */ +/** + * @deprecated the API will be removed in v2.0.0 (replaced with KeyManagementClient interface). + */ @Deprecated public interface KmsClient extends Serializable { diff --git a/api/src/main/java/org/apache/iceberg/exceptions/NoSuchIcebergViewException.java b/api/src/main/java/org/apache/iceberg/exceptions/NoSuchIcebergViewException.java new file mode 100644 index 000000000000..bc5da2aee280 --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/exceptions/NoSuchIcebergViewException.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.exceptions; + +import com.google.errorprone.annotations.FormatMethod; + +/** NoSuchIcebergViewException thrown when a view is found, but it is not an Iceberg view. */ +public class NoSuchIcebergViewException extends NoSuchViewException { + @FormatMethod + public NoSuchIcebergViewException(String message, Object... args) { + super(message, args); + } + + @FormatMethod + public static void check(boolean test, String message, Object... args) { + if (!test) { + throw new NoSuchIcebergViewException(message, args); + } + } +} diff --git a/api/src/main/java/org/apache/iceberg/expressions/BoundLiteralPredicate.java b/api/src/main/java/org/apache/iceberg/expressions/BoundLiteralPredicate.java index 02dc31c6a6c5..127d46e6a48f 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/BoundLiteralPredicate.java +++ b/api/src/main/java/org/apache/iceberg/expressions/BoundLiteralPredicate.java @@ -31,6 +31,7 @@ public class BoundLiteralPredicate extends BoundPredicate { Type.TypeID.LONG, Type.TypeID.DATE, Type.TypeID.TIME, + Type.TypeID.TIMESTAMP_NANO, Type.TypeID.TIMESTAMP); private static long toLong(Literal lit) { diff --git a/api/src/main/java/org/apache/iceberg/expressions/BoundReference.java b/api/src/main/java/org/apache/iceberg/expressions/BoundReference.java index ca51b1944c66..0ff73632b1d6 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/BoundReference.java +++ b/api/src/main/java/org/apache/iceberg/expressions/BoundReference.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.expressions; +import java.util.Locale; import org.apache.iceberg.Accessor; import org.apache.iceberg.StructLike; import org.apache.iceberg.types.Type; @@ -82,6 +83,7 @@ public Accessor accessor() { @Override public String toString() { - return String.format("ref(id=%d, accessor-type=%s)", field.fieldId(), accessor.type()); + return String.format( + Locale.ROOT, "ref(id=%d, accessor-type=%s)", field.fieldId(), accessor.type()); } } diff --git a/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java b/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java index bf72e03bc406..82d513ced7dd 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java +++ b/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java @@ -23,6 +23,7 @@ import java.time.ZoneOffset; import java.time.temporal.ChronoUnit; import java.util.List; +import java.util.Locale; import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.function.Function; @@ -36,6 +37,7 @@ import org.apache.iceberg.transforms.Transforms; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.DateTimeUtil; /** Expression utility methods. */ public class ExpressionUtil { @@ -52,6 +54,12 @@ public class ExpressionUtil { private static final Pattern TIMESTAMPTZ = Pattern.compile( "\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}(:\\d{2}(.\\d{1,9})?)?([-+]\\d{2}:\\d{2}|Z)"); + private static final Pattern TIMESTAMPNS = + Pattern.compile("\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}(:\\d{2}(.\\d{7,9})?)?"); + private static final Pattern TIMESTAMPTZNS = + Pattern.compile( + "\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}(:\\d{2}(.\\d{7,9})?)?([-+]\\d{2}:\\d{2}|Z)"); + static final int LONG_IN_PREDICATE_ABBREVIATION_THRESHOLD = 10; private static final int LONG_IN_PREDICATE_ABBREVIATION_MIN_GAIN = 5; @@ -493,8 +501,10 @@ private static List abbreviateValues(List sanitizedValues) { abbreviatedList.addAll(distinctValues); abbreviatedList.add( String.format( + Locale.ROOT, "... (%d values hidden, %d in total)", - sanitizedValues.size() - distinctValues.size(), sanitizedValues.size())); + sanitizedValues.size() - distinctValues.size(), + sanitizedValues.size())); return abbreviatedList; } } @@ -515,6 +525,8 @@ private static String sanitize(Type type, Object value, long now, int today) { return "(time)"; case TIMESTAMP: return sanitizeTimestamp((long) value, now); + case TIMESTAMP_NANO: + return sanitizeTimestamp(DateTimeUtil.nanosToMicros((long) value / 1000), now); case STRING: return sanitizeString((CharSequence) value, now, today); case BOOLEAN: @@ -536,6 +548,9 @@ private static String sanitize(Literal literal, long now, int today) { return sanitizeDate(((Literals.DateLiteral) literal).value(), today); } else if (literal instanceof Literals.TimestampLiteral) { return sanitizeTimestamp(((Literals.TimestampLiteral) literal).value(), now); + } else if (literal instanceof Literals.TimestampNanoLiteral) { + return sanitizeTimestamp( + DateTimeUtil.nanosToMicros(((Literals.TimestampNanoLiteral) literal).value()), now); } else if (literal instanceof Literals.TimeLiteral) { return "(time)"; } else if (literal instanceof Literals.IntegerLiteral) { @@ -594,6 +609,12 @@ private static String sanitizeString(CharSequence value, long now, int today) { if (DATE.matcher(value).matches()) { Literal date = Literal.of(value).to(Types.DateType.get()); return sanitizeDate(date.value(), today); + } else if (TIMESTAMPNS.matcher(value).matches()) { + Literal ts = Literal.of(value).to(Types.TimestampNanoType.withoutZone()); + return sanitizeTimestamp(DateTimeUtil.nanosToMicros(ts.value()), now); + } else if (TIMESTAMPTZNS.matcher(value).matches()) { + Literal ts = Literal.of(value).to(Types.TimestampNanoType.withZone()); + return sanitizeTimestamp(DateTimeUtil.nanosToMicros(ts.value()), now); } else if (TIMESTAMP.matcher(value).matches()) { Literal ts = Literal.of(value).to(Types.TimestampType.withoutZone()); return sanitizeTimestamp(ts.value(), now); @@ -615,7 +636,7 @@ private static String sanitizeString(CharSequence value, long now, int today) { private static String sanitizeSimpleString(CharSequence value) { // hash the value and return the hash as hex - return String.format("(hash-%08x)", HASH_FUNC.apply(value)); + return String.format(Locale.ROOT, "(hash-%08x)", HASH_FUNC.apply(value)); } private static PartitionSpec identitySpec(Schema schema, int... ids) { diff --git a/api/src/main/java/org/apache/iceberg/expressions/Expressions.java b/api/src/main/java/org/apache/iceberg/expressions/Expressions.java index f21a7705968b..deeba664ec07 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/Expressions.java +++ b/api/src/main/java/org/apache/iceberg/expressions/Expressions.java @@ -309,6 +309,18 @@ public static UnboundTerm transform(String name, Transform transfor return new UnboundTransform<>(ref(name), transform); } + /** + * Create a {@link Literal} from an Object. + * + * @param value a value + * @param Java type of value + * @return a Literal for the given value + * @throws IllegalArgumentException if the value has no literal implementation + */ + public static Literal lit(T value) { + return Literals.from(value); + } + public static UnboundAggregate count(String name) { return new UnboundAggregate<>(Operation.COUNT, ref(name)); } diff --git a/api/src/main/java/org/apache/iceberg/expressions/Literals.java b/api/src/main/java/org/apache/iceberg/expressions/Literals.java index 79d7190c49df..ee47035b1e72 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/Literals.java +++ b/api/src/main/java/org/apache/iceberg/expressions/Literals.java @@ -24,7 +24,6 @@ import java.nio.ByteBuffer; import java.time.Instant; import java.time.LocalDate; -import java.time.LocalDateTime; import java.time.LocalTime; import java.time.OffsetDateTime; import java.time.ZoneOffset; @@ -40,6 +39,7 @@ import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.ByteBuffers; +import org.apache.iceberg.util.DateTimeUtil; import org.apache.iceberg.util.NaNUtil; class Literals { @@ -299,6 +299,9 @@ public Literal to(Type type) { return (Literal) new TimeLiteral(value()); case TIMESTAMP: return (Literal) new TimestampLiteral(value()); + case TIMESTAMP_NANO: + // assume micros and convert to nanos to match the behavior in the timestamp case above + return new TimestampLiteral(value()).to(type); case DATE: if ((long) Integer.MAX_VALUE < value()) { return aboveMax(); @@ -437,11 +440,9 @@ public Literal to(Type type) { case TIMESTAMP: return (Literal) this; case DATE: - return (Literal) - new DateLiteral( - (int) - ChronoUnit.DAYS.between( - EPOCH_DAY, EPOCH.plus(value(), ChronoUnit.MICROS).toLocalDate())); + return (Literal) new DateLiteral(DateTimeUtil.microsToDays(value())); + case TIMESTAMP_NANO: + return (Literal) new TimestampNanoLiteral(DateTimeUtil.microsToNanos(value())); default: } return null; @@ -453,6 +454,32 @@ protected Type.TypeID typeId() { } } + static class TimestampNanoLiteral extends ComparableLiteral { + TimestampNanoLiteral(Long value) { + super(value); + } + + @Override + @SuppressWarnings("unchecked") + public Literal to(Type type) { + switch (type.typeId()) { + case DATE: + return (Literal) new DateLiteral(DateTimeUtil.nanosToDays(value())); + case TIMESTAMP: + return (Literal) new TimestampLiteral(DateTimeUtil.nanosToMicros(value())); + case TIMESTAMP_NANO: + return (Literal) this; + default: + } + return null; + } + + @Override + protected Type.TypeID typeId() { + return Type.TypeID.TIMESTAMP_NANO; + } + } + static class DecimalLiteral extends ComparableLiteral { DecimalLiteral(BigDecimal value) { super(value); @@ -502,19 +529,21 @@ public Literal to(Type type) { case TIMESTAMP: if (((Types.TimestampType) type).shouldAdjustToUTC()) { - long timestampMicros = - ChronoUnit.MICROS.between( - EPOCH, OffsetDateTime.parse(value(), DateTimeFormatter.ISO_DATE_TIME)); + long timestampMicros = DateTimeUtil.isoTimestamptzToMicros(value().toString()); return (Literal) new TimestampLiteral(timestampMicros); } else { - long timestampMicros = - ChronoUnit.MICROS.between( - EPOCH, - LocalDateTime.parse(value(), DateTimeFormatter.ISO_LOCAL_DATE_TIME) - .atOffset(ZoneOffset.UTC)); + long timestampMicros = DateTimeUtil.isoTimestampToMicros(value().toString()); return (Literal) new TimestampLiteral(timestampMicros); } + case TIMESTAMP_NANO: + if (((Types.TimestampNanoType) type).shouldAdjustToUTC()) { + return (Literal) + new TimestampNanoLiteral(DateTimeUtil.isoTimestamptzToNanos(value())); + } else { + return (Literal) new TimestampNanoLiteral(DateTimeUtil.isoTimestampToNanos(value())); + } + case STRING: return (Literal) this; diff --git a/api/src/main/java/org/apache/iceberg/expressions/StrictMetricsEvaluator.java b/api/src/main/java/org/apache/iceberg/expressions/StrictMetricsEvaluator.java index 4aee75c447d3..1a5a884f651a 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/StrictMetricsEvaluator.java +++ b/api/src/main/java/org/apache/iceberg/expressions/StrictMetricsEvaluator.java @@ -29,9 +29,7 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.Schema; import org.apache.iceberg.expressions.ExpressionVisitors.BoundExpressionVisitor; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.types.Conversions; -import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.StructType; import org.apache.iceberg.util.NaNUtil; @@ -51,7 +49,6 @@ * checks for NaN is necessary in order to not include files that may contain rows that don't match. */ public class StrictMetricsEvaluator { - private final Schema schema; private final StructType struct; private final Expression expr; @@ -60,7 +57,6 @@ public StrictMetricsEvaluator(Schema schema, Expression unbound) { } public StrictMetricsEvaluator(Schema schema, Expression unbound, boolean caseSensitive) { - this.schema = schema; this.struct = schema.asStruct(); this.expr = Binder.bind(struct, rewriteNot(unbound), caseSensitive); } @@ -144,8 +140,9 @@ public Boolean isNull(BoundReference ref) { // no need to check whether the field is required because binding evaluates that case // if the column has any non-null values, the expression does not match int id = ref.fieldId(); - Preconditions.checkNotNull( - struct.field(id), "Cannot filter by nested column: %s", schema.findField(id)); + if (isNestedColumn(id)) { + return ROWS_MIGHT_NOT_MATCH; + } if (containsNullsOnly(id)) { return ROWS_MUST_MATCH; @@ -159,8 +156,9 @@ public Boolean notNull(BoundReference ref) { // no need to check whether the field is required because binding evaluates that case // if the column has any null values, the expression does not match int id = ref.fieldId(); - Preconditions.checkNotNull( - struct.field(id), "Cannot filter by nested column: %s", schema.findField(id)); + if (isNestedColumn(id)) { + return ROWS_MIGHT_NOT_MATCH; + } if (nullCounts != null && nullCounts.containsKey(id) && nullCounts.get(id) == 0) { return ROWS_MUST_MATCH; @@ -199,15 +197,16 @@ public Boolean notNaN(BoundReference ref) { public Boolean lt(BoundReference ref, Literal lit) { // Rows must match when: <----------Min----Max---X-------> Integer id = ref.fieldId(); - Types.NestedField field = struct.field(id); - Preconditions.checkNotNull(field, "Cannot filter by nested column: %s", schema.findField(id)); + if (isNestedColumn(id)) { + return ROWS_MIGHT_NOT_MATCH; + } if (canContainNulls(id) || canContainNaNs(id)) { return ROWS_MIGHT_NOT_MATCH; } if (upperBounds != null && upperBounds.containsKey(id)) { - T upper = Conversions.fromByteBuffer(field.type(), upperBounds.get(id)); + T upper = Conversions.fromByteBuffer(ref.type(), upperBounds.get(id)); int cmp = lit.comparator().compare(upper, lit.value()); if (cmp < 0) { @@ -222,15 +221,16 @@ public Boolean lt(BoundReference ref, Literal lit) { public Boolean ltEq(BoundReference ref, Literal lit) { // Rows must match when: <----------Min----Max---X-------> Integer id = ref.fieldId(); - Types.NestedField field = struct.field(id); - Preconditions.checkNotNull(field, "Cannot filter by nested column: %s", schema.findField(id)); + if (isNestedColumn(id)) { + return ROWS_MIGHT_NOT_MATCH; + } if (canContainNulls(id) || canContainNaNs(id)) { return ROWS_MIGHT_NOT_MATCH; } if (upperBounds != null && upperBounds.containsKey(id)) { - T upper = Conversions.fromByteBuffer(field.type(), upperBounds.get(id)); + T upper = Conversions.fromByteBuffer(ref.type(), upperBounds.get(id)); int cmp = lit.comparator().compare(upper, lit.value()); if (cmp <= 0) { @@ -245,15 +245,16 @@ public Boolean ltEq(BoundReference ref, Literal lit) { public Boolean gt(BoundReference ref, Literal lit) { // Rows must match when: <-------X---Min----Max----------> Integer id = ref.fieldId(); - Types.NestedField field = struct.field(id); - Preconditions.checkNotNull(field, "Cannot filter by nested column: %s", schema.findField(id)); + if (isNestedColumn(id)) { + return ROWS_MIGHT_NOT_MATCH; + } if (canContainNulls(id) || canContainNaNs(id)) { return ROWS_MIGHT_NOT_MATCH; } if (lowerBounds != null && lowerBounds.containsKey(id)) { - T lower = Conversions.fromByteBuffer(field.type(), lowerBounds.get(id)); + T lower = Conversions.fromByteBuffer(ref.type(), lowerBounds.get(id)); if (NaNUtil.isNaN(lower)) { // NaN indicates unreliable bounds. See the StrictMetricsEvaluator docs for more. @@ -273,15 +274,16 @@ public Boolean gt(BoundReference ref, Literal lit) { public Boolean gtEq(BoundReference ref, Literal lit) { // Rows must match when: <-------X---Min----Max----------> Integer id = ref.fieldId(); - Types.NestedField field = struct.field(id); - Preconditions.checkNotNull(field, "Cannot filter by nested column: %s", schema.findField(id)); + if (isNestedColumn(id)) { + return ROWS_MIGHT_NOT_MATCH; + } if (canContainNulls(id) || canContainNaNs(id)) { return ROWS_MIGHT_NOT_MATCH; } if (lowerBounds != null && lowerBounds.containsKey(id)) { - T lower = Conversions.fromByteBuffer(field.type(), lowerBounds.get(id)); + T lower = Conversions.fromByteBuffer(ref.type(), lowerBounds.get(id)); if (NaNUtil.isNaN(lower)) { // NaN indicates unreliable bounds. See the StrictMetricsEvaluator docs for more. @@ -301,8 +303,9 @@ public Boolean gtEq(BoundReference ref, Literal lit) { public Boolean eq(BoundReference ref, Literal lit) { // Rows must match when Min == X == Max Integer id = ref.fieldId(); - Types.NestedField field = struct.field(id); - Preconditions.checkNotNull(field, "Cannot filter by nested column: %s", schema.findField(id)); + if (isNestedColumn(id)) { + return ROWS_MIGHT_NOT_MATCH; + } if (canContainNulls(id) || canContainNaNs(id)) { return ROWS_MIGHT_NOT_MATCH; @@ -319,7 +322,7 @@ public Boolean eq(BoundReference ref, Literal lit) { return ROWS_MIGHT_NOT_MATCH; } - T upper = Conversions.fromByteBuffer(field.type(), upperBounds.get(id)); + T upper = Conversions.fromByteBuffer(ref.type(), upperBounds.get(id)); cmp = lit.comparator().compare(upper, lit.value()); if (cmp != 0) { @@ -336,8 +339,9 @@ public Boolean eq(BoundReference ref, Literal lit) { public Boolean notEq(BoundReference ref, Literal lit) { // Rows must match when X < Min or Max < X because it is not in the range Integer id = ref.fieldId(); - Types.NestedField field = struct.field(id); - Preconditions.checkNotNull(field, "Cannot filter by nested column: %s", schema.findField(id)); + if (isNestedColumn(id)) { + return ROWS_MIGHT_NOT_MATCH; + } if (containsNullsOnly(id) || containsNaNsOnly(id)) { return ROWS_MUST_MATCH; @@ -358,7 +362,7 @@ public Boolean notEq(BoundReference ref, Literal lit) { } if (upperBounds != null && upperBounds.containsKey(id)) { - T upper = Conversions.fromByteBuffer(field.type(), upperBounds.get(id)); + T upper = Conversions.fromByteBuffer(ref.type(), upperBounds.get(id)); int cmp = lit.comparator().compare(upper, lit.value()); if (cmp < 0) { @@ -372,8 +376,9 @@ public Boolean notEq(BoundReference ref, Literal lit) { @Override public Boolean in(BoundReference ref, Set literalSet) { Integer id = ref.fieldId(); - Types.NestedField field = struct.field(id); - Preconditions.checkNotNull(field, "Cannot filter by nested column: %s", schema.findField(id)); + if (isNestedColumn(id)) { + return ROWS_MIGHT_NOT_MATCH; + } if (canContainNulls(id) || canContainNaNs(id)) { return ROWS_MIGHT_NOT_MATCH; @@ -390,7 +395,7 @@ public Boolean in(BoundReference ref, Set literalSet) { } // check if the upper bound is in the set - T upper = Conversions.fromByteBuffer(field.type(), upperBounds.get(id)); + T upper = Conversions.fromByteBuffer(ref.type(), upperBounds.get(id)); if (!literalSet.contains(upper)) { return ROWS_MIGHT_NOT_MATCH; } @@ -411,8 +416,9 @@ public Boolean in(BoundReference ref, Set literalSet) { @Override public Boolean notIn(BoundReference ref, Set literalSet) { Integer id = ref.fieldId(); - Types.NestedField field = struct.field(id); - Preconditions.checkNotNull(field, "Cannot filter by nested column: %s", schema.findField(id)); + if (isNestedColumn(id)) { + return ROWS_MIGHT_NOT_MATCH; + } if (containsNullsOnly(id) || containsNaNsOnly(id)) { return ROWS_MUST_MATCH; @@ -439,7 +445,7 @@ public Boolean notIn(BoundReference ref, Set literalSet) { } if (upperBounds != null && upperBounds.containsKey(id)) { - T upper = Conversions.fromByteBuffer(field.type(), upperBounds.get(id)); + T upper = Conversions.fromByteBuffer(ref.type(), upperBounds.get(id)); literals = literals.stream() .filter(v -> ref.comparator().compare(upper, v) >= 0) @@ -466,6 +472,10 @@ public Boolean notStartsWith(BoundReference ref, Literal lit) { return ROWS_MIGHT_NOT_MATCH; } + private boolean isNestedColumn(int id) { + return struct.field(id) == null; + } + private boolean canContainNulls(Integer id) { return nullCounts == null || (nullCounts.containsKey(id) && nullCounts.get(id) > 0); } diff --git a/api/src/main/java/org/apache/iceberg/io/BulkDeletionFailureException.java b/api/src/main/java/org/apache/iceberg/io/BulkDeletionFailureException.java index 535be5f64ec8..4f89d462fe94 100644 --- a/api/src/main/java/org/apache/iceberg/io/BulkDeletionFailureException.java +++ b/api/src/main/java/org/apache/iceberg/io/BulkDeletionFailureException.java @@ -18,11 +18,13 @@ */ package org.apache.iceberg.io; +import java.util.Locale; + public class BulkDeletionFailureException extends RuntimeException { private final int numberFailedObjects; public BulkDeletionFailureException(int numberFailedObjects) { - super(String.format("Failed to delete %d files", numberFailedObjects)); + super(String.format(Locale.ROOT, "Failed to delete %d files", numberFailedObjects)); this.numberFailedObjects = numberFailedObjects; } diff --git a/api/src/main/java/org/apache/iceberg/transforms/Bucket.java b/api/src/main/java/org/apache/iceberg/transforms/Bucket.java index 912bcd271725..0e4e782cc110 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Bucket.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Bucket.java @@ -33,6 +33,7 @@ import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.BucketUtil; +import org.apache.iceberg.util.DateTimeUtil; import org.apache.iceberg.util.SerializableFunction; class Bucket implements Transform, Serializable { @@ -63,6 +64,8 @@ static & SerializableFunction> B get( case FIXED: case BINARY: return (B) new BucketByteBuffer(numBuckets); + case TIMESTAMP_NANO: + return (B) new BucketTimestampNano(numBuckets); case UUID: return (B) new BucketUUID(numBuckets); default: @@ -107,6 +110,7 @@ public boolean canTransform(Type type) { case DATE: case TIME: case TIMESTAMP: + case TIMESTAMP_NANO: case STRING: case BINARY: case FIXED: @@ -214,6 +218,20 @@ protected int hash(Long value) { } } + // In order to bucket TimestampNano the same as Timestamp, convert to micros before hashing. + private static class BucketTimestampNano extends Bucket + implements SerializableFunction { + + private BucketTimestampNano(int numBuckets) { + super(numBuckets); + } + + @Override + protected int hash(Long nanos) { + return BucketUtil.hash(DateTimeUtil.nanosToMicros(nanos)); + } + } + private static class BucketString extends Bucket implements SerializableFunction { diff --git a/api/src/main/java/org/apache/iceberg/transforms/Dates.java b/api/src/main/java/org/apache/iceberg/transforms/Dates.java index 3d26b542be7b..88db16797867 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Dates.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Dates.java @@ -97,6 +97,10 @@ public Type getResultType(Type sourceType) { return Types.IntegerType.get(); } + ChronoUnit granularity() { + return granularity; + } + @Override public boolean preservesOrder() { return true; @@ -109,11 +113,11 @@ public boolean satisfiesOrderOf(Transform other) { } if (other instanceof Dates) { - // test the granularity, in days. day(ts) => 1 day, months(ts) => 30 days, and day satisfies - // the order of months - Dates otherTransform = (Dates) other; - return granularity.getDuration().toDays() - <= otherTransform.granularity.getDuration().toDays(); + return TransformUtil.satisfiesOrderOf(granularity, ((Dates) other).granularity()); + } else if (other instanceof Timestamps) { + return TransformUtil.satisfiesOrderOf(granularity, ((Timestamps) other).granularity()); + } else if (other instanceof TimeTransform) { + return TransformUtil.satisfiesOrderOf(granularity, ((TimeTransform) other).granularity()); } return false; diff --git a/api/src/main/java/org/apache/iceberg/transforms/Days.java b/api/src/main/java/org/apache/iceberg/transforms/Days.java index f69d5d6110ed..e2b829b86662 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Days.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Days.java @@ -19,6 +19,7 @@ package org.apache.iceberg.transforms; import java.io.ObjectStreamException; +import java.time.temporal.ChronoUnit; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; @@ -31,38 +32,19 @@ static Days get() { } @Override - @SuppressWarnings("unchecked") - protected Transform toEnum(Type type) { - switch (type.typeId()) { - case DATE: - return (Transform) Dates.DAY; - case TIMESTAMP: - return (Transform) Timestamps.DAY; - default: - throw new IllegalArgumentException("Unsupported type: " + type); - } + protected ChronoUnit granularity() { + return ChronoUnit.DAYS; } @Override - public Type getResultType(Type sourceType) { - return Types.DateType.get(); + protected Transform toEnum(Type type) { + return (Transform) + fromSourceType(type, Dates.DAY, Timestamps.MICROS_TO_DAY, Timestamps.NANOS_TO_DAY); } @Override - public boolean satisfiesOrderOf(Transform other) { - if (this == other) { - return true; - } - - if (other instanceof Timestamps) { - return Timestamps.DAY.satisfiesOrderOf(other); - } else if (other instanceof Dates) { - return Dates.DAY.satisfiesOrderOf(other); - } else if (other instanceof Days || other instanceof Months || other instanceof Years) { - return true; - } - - return false; + public Type getResultType(Type sourceType) { + return Types.DateType.get(); } @Override diff --git a/api/src/main/java/org/apache/iceberg/transforms/Hours.java b/api/src/main/java/org/apache/iceberg/transforms/Hours.java index afc14516f3cd..2ff79f6a66a7 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Hours.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Hours.java @@ -19,6 +19,7 @@ package org.apache.iceberg.transforms; import java.io.ObjectStreamException; +import java.time.temporal.ChronoUnit; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; @@ -30,19 +31,21 @@ static Hours get() { return (Hours) INSTANCE; } + @Override + protected ChronoUnit granularity() { + return ChronoUnit.HOURS; + } + @Override @SuppressWarnings("unchecked") protected Transform toEnum(Type type) { - if (type.typeId() == Type.TypeID.TIMESTAMP) { - return (Transform) Timestamps.HOUR; - } - - throw new IllegalArgumentException("Unsupported type: " + type); + return (Transform) + fromSourceType(type, null, Timestamps.MICROS_TO_HOUR, Timestamps.NANOS_TO_HOUR); } @Override public boolean canTransform(Type type) { - return type.typeId() == Type.TypeID.TIMESTAMP; + return type.typeId() == Type.TypeID.TIMESTAMP || type.typeId() == Type.TypeID.TIMESTAMP_NANO; } @Override @@ -50,24 +53,6 @@ public Type getResultType(Type sourceType) { return Types.IntegerType.get(); } - @Override - public boolean satisfiesOrderOf(Transform other) { - if (this == other) { - return true; - } - - if (other instanceof Timestamps) { - return other == Timestamps.HOUR; - } else if (other instanceof Hours - || other instanceof Days - || other instanceof Months - || other instanceof Years) { - return true; - } - - return false; - } - @Override public String toHumanString(Type alwaysInt, Integer value) { return value != null ? TransformUtil.humanHour(value) : "null"; diff --git a/api/src/main/java/org/apache/iceberg/transforms/Months.java b/api/src/main/java/org/apache/iceberg/transforms/Months.java index 8fa4d42385f7..73ec50e5dd9a 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Months.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Months.java @@ -19,6 +19,7 @@ package org.apache.iceberg.transforms; import java.io.ObjectStreamException; +import java.time.temporal.ChronoUnit; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; @@ -31,38 +32,19 @@ static Months get() { } @Override - @SuppressWarnings("unchecked") - protected Transform toEnum(Type type) { - switch (type.typeId()) { - case DATE: - return (Transform) Dates.MONTH; - case TIMESTAMP: - return (Transform) Timestamps.MONTH; - default: - throw new IllegalArgumentException("Unsupported type: " + type); - } + protected ChronoUnit granularity() { + return ChronoUnit.MONTHS; } @Override - public Type getResultType(Type sourceType) { - return Types.IntegerType.get(); + protected Transform toEnum(Type type) { + return (Transform) + fromSourceType(type, Dates.MONTH, Timestamps.MICROS_TO_MONTH, Timestamps.NANOS_TO_MONTH); } @Override - public boolean satisfiesOrderOf(Transform other) { - if (this == other) { - return true; - } - - if (other instanceof Timestamps) { - return Timestamps.MONTH.satisfiesOrderOf(other); - } else if (other instanceof Dates) { - return Dates.MONTH.satisfiesOrderOf(other); - } else if (other instanceof Months || other instanceof Years) { - return true; - } - - return false; + public Type getResultType(Type sourceType) { + return Types.IntegerType.get(); } @Override diff --git a/api/src/main/java/org/apache/iceberg/transforms/PartitionSpecVisitor.java b/api/src/main/java/org/apache/iceberg/transforms/PartitionSpecVisitor.java index e4796478bf28..0d80ef88a296 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/PartitionSpecVisitor.java +++ b/api/src/main/java/org/apache/iceberg/transforms/PartitionSpecVisitor.java @@ -122,16 +122,23 @@ static R visit(Schema schema, PartitionField field, PartitionSpecVisitor int width = ((Truncate) transform).width(); return visitor.truncate(field.fieldId(), sourceName, field.sourceId(), width); } else if (transform == Dates.YEAR - || transform == Timestamps.YEAR + || transform == Timestamps.MICROS_TO_YEAR + || transform == Timestamps.NANOS_TO_YEAR || transform instanceof Years) { return visitor.year(field.fieldId(), sourceName, field.sourceId()); } else if (transform == Dates.MONTH - || transform == Timestamps.MONTH + || transform == Timestamps.MICROS_TO_MONTH + || transform == Timestamps.NANOS_TO_MONTH || transform instanceof Months) { return visitor.month(field.fieldId(), sourceName, field.sourceId()); - } else if (transform == Dates.DAY || transform == Timestamps.DAY || transform instanceof Days) { + } else if (transform == Dates.DAY + || transform == Timestamps.MICROS_TO_DAY + || transform == Timestamps.NANOS_TO_DAY + || transform instanceof Days) { return visitor.day(field.fieldId(), sourceName, field.sourceId()); - } else if (transform == Timestamps.HOUR || transform instanceof Hours) { + } else if (transform == Timestamps.MICROS_TO_HOUR + || transform == Timestamps.NANOS_TO_HOUR + || transform instanceof Hours) { return visitor.hour(field.fieldId(), sourceName, field.sourceId()); } else if (transform instanceof VoidTransform) { return visitor.alwaysNull(field.fieldId(), sourceName, field.sourceId()); diff --git a/api/src/main/java/org/apache/iceberg/transforms/SortOrderVisitor.java b/api/src/main/java/org/apache/iceberg/transforms/SortOrderVisitor.java index 680e095270fb..62cc9d3cdb33 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/SortOrderVisitor.java +++ b/api/src/main/java/org/apache/iceberg/transforms/SortOrderVisitor.java @@ -85,21 +85,26 @@ static List visit(SortOrder sortOrder, SortOrderVisitor visitor) { visitor.truncate( sourceName, field.sourceId(), width, field.direction(), field.nullOrder())); } else if (transform == Dates.YEAR - || transform == Timestamps.YEAR + || transform == Timestamps.MICROS_TO_YEAR + || transform == Timestamps.NANOS_TO_YEAR || transform instanceof Years) { results.add( visitor.year(sourceName, field.sourceId(), field.direction(), field.nullOrder())); } else if (transform == Dates.MONTH - || transform == Timestamps.MONTH + || transform == Timestamps.MICROS_TO_MONTH + || transform == Timestamps.NANOS_TO_MONTH || transform instanceof Months) { results.add( visitor.month(sourceName, field.sourceId(), field.direction(), field.nullOrder())); } else if (transform == Dates.DAY - || transform == Timestamps.DAY + || transform == Timestamps.MICROS_TO_DAY + || transform == Timestamps.NANOS_TO_DAY || transform instanceof Days) { results.add( visitor.day(sourceName, field.sourceId(), field.direction(), field.nullOrder())); - } else if (transform == Timestamps.HOUR || transform instanceof Hours) { + } else if (transform == Timestamps.MICROS_TO_HOUR + || transform == Timestamps.NANOS_TO_HOUR + || transform instanceof Hours) { results.add( visitor.hour(sourceName, field.sourceId(), field.direction(), field.nullOrder())); } else if (transform instanceof UnknownTransform) { diff --git a/api/src/main/java/org/apache/iceberg/transforms/TimeTransform.java b/api/src/main/java/org/apache/iceberg/transforms/TimeTransform.java index 01ea8130aa60..c348fda52b02 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/TimeTransform.java +++ b/api/src/main/java/org/apache/iceberg/transforms/TimeTransform.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.transforms; +import java.time.temporal.ChronoUnit; import org.apache.iceberg.expressions.BoundPredicate; import org.apache.iceberg.expressions.BoundTransform; import org.apache.iceberg.expressions.UnboundPredicate; @@ -25,6 +26,24 @@ import org.apache.iceberg.util.SerializableFunction; abstract class TimeTransform implements Transform { + protected static R fromSourceType(Type type, R dateResult, R microsResult, R nanosResult) { + switch (type.typeId()) { + case DATE: + if (dateResult != null) { + return dateResult; + } + break; + case TIMESTAMP: + return microsResult; + case TIMESTAMP_NANO: + return nanosResult; + } + + throw new IllegalArgumentException("Unsupported type: " + type); + } + + protected abstract ChronoUnit granularity(); + protected abstract Transform toEnum(Type type); @Override @@ -37,9 +56,29 @@ public boolean preservesOrder() { return true; } + @Override + public boolean satisfiesOrderOf(Transform other) { + if (this == other) { + return true; + } + + if (other instanceof Dates) { + return TransformUtil.satisfiesOrderOf(granularity(), ((Dates) other).granularity()); + } else if (other instanceof Timestamps) { + return TransformUtil.satisfiesOrderOf(granularity(), ((Timestamps) other).granularity()); + } else if (other instanceof TimeTransform) { + return TransformUtil.satisfiesOrderOf( + granularity(), ((TimeTransform) other).granularity()); + } + + return false; + } + @Override public boolean canTransform(Type type) { - return type.typeId() == Type.TypeID.DATE || type.typeId() == Type.TypeID.TIMESTAMP; + return type.typeId() == Type.TypeID.DATE + || type.typeId() == Type.TypeID.TIMESTAMP + || type.typeId() == Type.TypeID.TIMESTAMP_NANO; } @Override diff --git a/api/src/main/java/org/apache/iceberg/transforms/Timestamps.java b/api/src/main/java/org/apache/iceberg/transforms/Timestamps.java index b5b50e9d42b2..8b8c2ca0a96b 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Timestamps.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Timestamps.java @@ -32,53 +32,29 @@ import org.apache.iceberg.util.SerializableFunction; enum Timestamps implements Transform { - YEAR(ChronoUnit.YEARS, "year"), - MONTH(ChronoUnit.MONTHS, "month"), - DAY(ChronoUnit.DAYS, "day"), - HOUR(ChronoUnit.HOURS, "hour"); + MICROS_TO_YEAR(ChronoUnit.YEARS, "year", MicrosToYears.INSTANCE), + MICROS_TO_MONTH(ChronoUnit.MONTHS, "month", MicrosToMonths.INSTANCE), + MICROS_TO_DAY(ChronoUnit.DAYS, "day", MicrosToDays.INSTANCE), + MICROS_TO_HOUR(ChronoUnit.HOURS, "hour", MicrosToHours.INSTANCE), - @Immutable - static class Apply implements SerializableFunction { - private final ChronoUnit granularity; - - Apply(ChronoUnit granularity) { - this.granularity = granularity; - } - - @Override - public Integer apply(Long timestampMicros) { - if (timestampMicros == null) { - return null; - } - - switch (granularity) { - case YEARS: - return DateTimeUtil.microsToYears(timestampMicros); - case MONTHS: - return DateTimeUtil.microsToMonths(timestampMicros); - case DAYS: - return DateTimeUtil.microsToDays(timestampMicros); - case HOURS: - return DateTimeUtil.microsToHours(timestampMicros); - default: - throw new UnsupportedOperationException("Unsupported time unit: " + granularity); - } - } - } + NANOS_TO_YEAR(ChronoUnit.YEARS, "year", NanosToYears.INSTANCE), + NANOS_TO_MONTH(ChronoUnit.MONTHS, "month", NanosToMonths.INSTANCE), + NANOS_TO_DAY(ChronoUnit.DAYS, "day", NanosToDays.INSTANCE), + NANOS_TO_HOUR(ChronoUnit.HOURS, "hour", NanosToHours.INSTANCE); private final ChronoUnit granularity; private final String name; - private final Apply apply; + private final SerializableFunction apply; - Timestamps(ChronoUnit granularity, String name) { - this.granularity = granularity; + Timestamps(ChronoUnit granularity, String name, SerializableFunction apply) { this.name = name; - this.apply = new Apply(granularity); + this.granularity = granularity; + this.apply = apply; } @Override - public Integer apply(Long timestampMicros) { - return apply.apply(timestampMicros); + public Integer apply(Long timestamp) { + return apply.apply(timestamp); } @Override @@ -89,7 +65,7 @@ public SerializableFunction bind(Type type) { @Override public boolean canTransform(Type type) { - return type.typeId() == Type.TypeID.TIMESTAMP; + return type.typeId() == Type.TypeID.TIMESTAMP || type.typeId() == Type.TypeID.TIMESTAMP_NANO; } @Override @@ -100,6 +76,10 @@ public Type getResultType(Type sourceType) { return Types.IntegerType.get(); } + ChronoUnit granularity() { + return granularity; + } + @Override public boolean preservesOrder() { return true; @@ -111,12 +91,12 @@ public boolean satisfiesOrderOf(Transform other) { return true; } - if (other instanceof Timestamps) { - // test the granularity, in hours. hour(ts) => 1 hour, day(ts) => 24 hours, and hour satisfies - // the order of day - Timestamps otherTransform = (Timestamps) other; - return granularity.getDuration().toHours() - <= otherTransform.granularity.getDuration().toHours(); + if (other instanceof Dates) { + return TransformUtil.satisfiesOrderOf(granularity, ((Dates) other).granularity()); + } else if (other instanceof Timestamps) { + return TransformUtil.satisfiesOrderOf(granularity, ((Timestamps) other).granularity()); + } else if (other instanceof TimeTransform) { + return TransformUtil.satisfiesOrderOf(granularity, ((TimeTransform) other).granularity()); } return false; @@ -197,4 +177,116 @@ public String toString() { public String dedupName() { return "time"; } + + @Immutable + static class MicrosToYears implements SerializableFunction { + static final MicrosToYears INSTANCE = new MicrosToYears(); + + @Override + public Integer apply(Long micros) { + if (micros == null) { + return null; + } + + return DateTimeUtil.microsToYears(micros); + } + } + + @Immutable + static class MicrosToMonths implements SerializableFunction { + static final MicrosToMonths INSTANCE = new MicrosToMonths(); + + @Override + public Integer apply(Long micros) { + if (micros == null) { + return null; + } + + return DateTimeUtil.microsToMonths(micros); + } + } + + @Immutable + static class MicrosToDays implements SerializableFunction { + static final MicrosToDays INSTANCE = new MicrosToDays(); + + @Override + public Integer apply(Long micros) { + if (micros == null) { + return null; + } + + return DateTimeUtil.microsToDays(micros); + } + } + + @Immutable + static class MicrosToHours implements SerializableFunction { + static final MicrosToHours INSTANCE = new MicrosToHours(); + + @Override + public Integer apply(Long micros) { + if (micros == null) { + return null; + } + + return DateTimeUtil.microsToHours(micros); + } + } + + @Immutable + static class NanosToYears implements SerializableFunction { + static final NanosToYears INSTANCE = new NanosToYears(); + + @Override + public Integer apply(Long nanos) { + if (nanos == null) { + return null; + } + + return DateTimeUtil.nanosToYears(nanos); + } + } + + @Immutable + static class NanosToMonths implements SerializableFunction { + static final NanosToMonths INSTANCE = new NanosToMonths(); + + @Override + public Integer apply(Long nanos) { + if (nanos == null) { + return null; + } + + return DateTimeUtil.nanosToMonths(nanos); + } + } + + @Immutable + static class NanosToDays implements SerializableFunction { + static final NanosToDays INSTANCE = new NanosToDays(); + + @Override + public Integer apply(Long nanos) { + if (nanos == null) { + return null; + } + + return DateTimeUtil.nanosToDays(nanos); + } + } + + @Immutable + static class NanosToHours implements SerializableFunction { + static final NanosToHours INSTANCE = new NanosToHours(); + + @Override + public Integer apply(Long nanos) { + if (nanos == null) { + return null; + } + + return DateTimeUtil.nanosToHours(nanos); + } + } } diff --git a/api/src/main/java/org/apache/iceberg/transforms/Transform.java b/api/src/main/java/org/apache/iceberg/transforms/Transform.java index 5a56b672b1b1..78312b58b12f 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Transform.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Transform.java @@ -181,6 +181,12 @@ default String toHumanString(Type type, T value) { } else { return TransformUtil.humanTimestampWithoutZone((Long) value); } + case TIMESTAMP_NANO: + if (((Types.TimestampNanoType) type).shouldAdjustToUTC()) { + return TransformUtil.humanTimestampNanoWithZone((Long) value); + } else { + return TransformUtil.humanTimestampNanoWithoutZone((Long) value); + } case FIXED: case BINARY: if (value instanceof ByteBuffer) { diff --git a/api/src/main/java/org/apache/iceberg/transforms/TransformUtil.java b/api/src/main/java/org/apache/iceberg/transforms/TransformUtil.java index 53bc23a49888..710019225e09 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/TransformUtil.java +++ b/api/src/main/java/org/apache/iceberg/transforms/TransformUtil.java @@ -26,6 +26,8 @@ import java.time.ZoneOffset; import java.time.temporal.ChronoUnit; import java.util.Base64; +import java.util.Locale; +import org.apache.iceberg.util.DateTimeUtil; class TransformUtil { @@ -35,19 +37,25 @@ private TransformUtil() {} private static final int EPOCH_YEAR = EPOCH.getYear(); static String humanYear(int yearOrdinal) { - return String.format("%04d", EPOCH_YEAR + yearOrdinal); + return String.format(Locale.ROOT, "%04d", EPOCH_YEAR + yearOrdinal); } static String humanMonth(int monthOrdinal) { return String.format( + Locale.ROOT, "%04d-%02d", - EPOCH_YEAR + Math.floorDiv(monthOrdinal, 12), 1 + Math.floorMod(monthOrdinal, 12)); + EPOCH_YEAR + Math.floorDiv(monthOrdinal, 12), + 1 + Math.floorMod(monthOrdinal, 12)); } static String humanDay(int dayOrdinal) { OffsetDateTime day = EPOCH.plusDays(dayOrdinal); return String.format( - "%04d-%02d-%02d", day.getYear(), day.getMonth().getValue(), day.getDayOfMonth()); + Locale.ROOT, + "%04d-%02d-%02d", + day.getYear(), + day.getMonth().getValue(), + day.getDayOfMonth()); } static String humanTime(Long microsFromMidnight) { @@ -55,22 +63,40 @@ static String humanTime(Long microsFromMidnight) { } static String humanTimestampWithZone(Long timestampMicros) { - return ChronoUnit.MICROS.addTo(EPOCH, timestampMicros).toString(); + return DateTimeUtil.microsToIsoTimestamptz(timestampMicros); } static String humanTimestampWithoutZone(Long timestampMicros) { - return ChronoUnit.MICROS.addTo(EPOCH, timestampMicros).toLocalDateTime().toString(); + return DateTimeUtil.microsToIsoTimestamp(timestampMicros); + } + + static String humanTimestampNanoWithZone(Long timestampNanos) { + return DateTimeUtil.nanosToIsoTimestamptz(timestampNanos); + } + + static String humanTimestampNanoWithoutZone(Long timestampNanos) { + return DateTimeUtil.nanosToIsoTimestamp(timestampNanos); } static String humanHour(int hourOrdinal) { OffsetDateTime time = EPOCH.plusHours(hourOrdinal); return String.format( + Locale.ROOT, "%04d-%02d-%02d-%02d", - time.getYear(), time.getMonth().getValue(), time.getDayOfMonth(), time.getHour()); + time.getYear(), + time.getMonth().getValue(), + time.getDayOfMonth(), + time.getHour()); } static String base64encode(ByteBuffer buffer) { // use direct encoding because all of the encoded bytes are in ASCII return StandardCharsets.ISO_8859_1.decode(Base64.getEncoder().encode(buffer)).toString(); } + + static boolean satisfiesOrderOf(ChronoUnit leftGranularity, ChronoUnit rightGranularity) { + // test the granularity, in hours. hour(ts) => 1 hour, day(ts) => 24 hours, and hour satisfies + // the order of day + return leftGranularity.getDuration().toHours() <= rightGranularity.getDuration().toHours(); + } } diff --git a/api/src/main/java/org/apache/iceberg/transforms/Transforms.java b/api/src/main/java/org/apache/iceberg/transforms/Transforms.java index a1ce33ddd6da..aacd4d430069 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Transforms.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Transforms.java @@ -23,7 +23,6 @@ import java.util.regex.Pattern; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.types.Type; /** @@ -68,6 +67,10 @@ private Transforms() {} return new UnknownTransform<>(transform); } + /** + * @deprecated use {@link #identity()} instead; will be removed in 2.0.0 + */ + @Deprecated public static Transform fromString(Type type, String transform) { Matcher widthMatcher = HAS_WIDTH.matcher(transform); if (widthMatcher.matches()) { @@ -80,22 +83,20 @@ private Transforms() {} } } - if (transform.equalsIgnoreCase("identity")) { - return Identity.get(type); - } - - try { - if (type.typeId() == Type.TypeID.TIMESTAMP) { - return Timestamps.valueOf(transform.toUpperCase(Locale.ENGLISH)); - } else if (type.typeId() == Type.TypeID.DATE) { - return Dates.valueOf(transform.toUpperCase(Locale.ENGLISH)); - } - } catch (IllegalArgumentException ignored) { - // fall through to return unknown transform - } - - if (transform.equalsIgnoreCase("void")) { - return VoidTransform.get(); + String lowerTransform = transform.toLowerCase(Locale.ENGLISH); + switch (lowerTransform) { + case "identity": + return Identity.get(type); + case "year": + return Years.get().toEnum(type); + case "month": + return Months.get().toEnum(type); + case "day": + return Days.get().toEnum(type); + case "hour": + return Hours.get().toEnum(type); + case "void": + return VoidTransform.get(); } return new UnknownTransform<>(transform); @@ -125,14 +126,7 @@ public static Transform identity(Type type) { @Deprecated @SuppressWarnings("unchecked") public static Transform year(Type type) { - switch (type.typeId()) { - case DATE: - return (Transform) Dates.YEAR; - case TIMESTAMP: - return (Transform) Timestamps.YEAR; - default: - throw new IllegalArgumentException("Cannot partition type " + type + " by year"); - } + return (Transform) Years.get().toEnum(type); } /** @@ -146,14 +140,7 @@ public static Transform year(Type type) { @Deprecated @SuppressWarnings("unchecked") public static Transform month(Type type) { - switch (type.typeId()) { - case DATE: - return (Transform) Dates.MONTH; - case TIMESTAMP: - return (Transform) Timestamps.MONTH; - default: - throw new IllegalArgumentException("Cannot partition type " + type + " by month"); - } + return (Transform) Months.get().toEnum(type); } /** @@ -167,14 +154,7 @@ public static Transform month(Type type) { @Deprecated @SuppressWarnings("unchecked") public static Transform day(Type type) { - switch (type.typeId()) { - case DATE: - return (Transform) Dates.DAY; - case TIMESTAMP: - return (Transform) Timestamps.DAY; - default: - throw new IllegalArgumentException("Cannot partition type " + type + " by day"); - } + return (Transform) Days.get().toEnum(type); } /** @@ -188,9 +168,7 @@ public static Transform day(Type type) { @Deprecated @SuppressWarnings("unchecked") public static Transform hour(Type type) { - Preconditions.checkArgument( - type.typeId() == Type.TypeID.TIMESTAMP, "Cannot partition type %s by hour", type); - return (Transform) Timestamps.HOUR; + return (Transform) Hours.get().toEnum(type); } /** diff --git a/api/src/main/java/org/apache/iceberg/transforms/Truncate.java b/api/src/main/java/org/apache/iceberg/transforms/Truncate.java index 670c6002a97a..a111e4ca394b 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Truncate.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Truncate.java @@ -43,7 +43,9 @@ static Truncate get(int width) { return new Truncate<>(width); } - /** @deprecated will be removed in 2.0.0 */ + /** + * @deprecated will be removed in 2.0.0 + */ @Deprecated @SuppressWarnings("unchecked") static & SerializableFunction> R get(Type type, int width) { diff --git a/api/src/main/java/org/apache/iceberg/transforms/Years.java b/api/src/main/java/org/apache/iceberg/transforms/Years.java index 6c1eee578506..2920a37dc692 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Years.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Years.java @@ -19,6 +19,7 @@ package org.apache.iceberg.transforms; import java.io.ObjectStreamException; +import java.time.temporal.ChronoUnit; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; @@ -31,38 +32,19 @@ static Years get() { } @Override - @SuppressWarnings("unchecked") - protected Transform toEnum(Type type) { - switch (type.typeId()) { - case DATE: - return (Transform) Dates.YEAR; - case TIMESTAMP: - return (Transform) Timestamps.YEAR; - default: - throw new IllegalArgumentException("Unsupported type: " + type); - } + protected ChronoUnit granularity() { + return ChronoUnit.YEARS; } @Override - public Type getResultType(Type sourceType) { - return Types.IntegerType.get(); + protected Transform toEnum(Type type) { + return (Transform) + fromSourceType(type, Dates.YEAR, Timestamps.MICROS_TO_YEAR, Timestamps.NANOS_TO_YEAR); } @Override - public boolean satisfiesOrderOf(Transform other) { - if (this == other) { - return true; - } - - if (other instanceof Timestamps) { - return Timestamps.YEAR.satisfiesOrderOf(other); - } else if (other instanceof Dates) { - return Dates.YEAR.satisfiesOrderOf(other); - } else if (other instanceof Years) { - return true; - } - - return false; + public Type getResultType(Type sourceType) { + return Types.IntegerType.get(); } @Override diff --git a/api/src/main/java/org/apache/iceberg/types/Comparators.java b/api/src/main/java/org/apache/iceberg/types/Comparators.java index a803afac104f..98416c6943db 100644 --- a/api/src/main/java/org/apache/iceberg/types/Comparators.java +++ b/api/src/main/java/org/apache/iceberg/types/Comparators.java @@ -41,6 +41,8 @@ private Comparators() {} .put(Types.TimeType.get(), Comparator.naturalOrder()) .put(Types.TimestampType.withZone(), Comparator.naturalOrder()) .put(Types.TimestampType.withoutZone(), Comparator.naturalOrder()) + .put(Types.TimestampNanoType.withZone(), Comparator.naturalOrder()) + .put(Types.TimestampNanoType.withoutZone(), Comparator.naturalOrder()) .put(Types.StringType.get(), Comparators.charSequences()) .put(Types.UUIDType.get(), Comparator.naturalOrder()) .put(Types.BinaryType.get(), Comparators.unsignedBytes()) @@ -321,9 +323,9 @@ private CharSeqComparator() {} * represented using two Java characters (using UTF-16 surrogate pairs). Character by character * comparison may yield incorrect results while comparing a 4 byte UTF-8 character to a java * char. Character by character comparison works as expected if both characters are <= 3 byte - * UTF-8 character or both characters are 4 byte UTF-8 characters. - * isCharInUTF16HighSurrogateRange method detects a 4-byte character and considers that - * character to be lexicographically greater than any 3 byte or lower UTF-8 character. + * UTF-8 character or both characters are 4 byte UTF-8 characters. isCharHighSurrogate method + * detects a high surrogate (4-byte character) and considers that character to be + * lexicographically greater than any 3 byte or lower UTF-8 character. */ @Override public int compare(CharSequence s1, CharSequence s2) { diff --git a/api/src/main/java/org/apache/iceberg/types/Conversions.java b/api/src/main/java/org/apache/iceberg/types/Conversions.java index 1d2539514954..e18c7b4362e6 100644 --- a/api/src/main/java/org/apache/iceberg/types/Conversions.java +++ b/api/src/main/java/org/apache/iceberg/types/Conversions.java @@ -97,6 +97,7 @@ public static ByteBuffer toByteBuffer(Type.TypeID typeId, Object value) { case LONG: case TIME: case TIMESTAMP: + case TIMESTAMP_NANO: return ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN).putLong(0, (long) value); case FLOAT: return ByteBuffer.allocate(4).order(ByteOrder.LITTLE_ENDIAN).putFloat(0, (float) value); @@ -146,6 +147,7 @@ private static Object internalFromByteBuffer(Type type, ByteBuffer buffer) { case LONG: case TIME: case TIMESTAMP: + case TIMESTAMP_NANO: if (tmp.remaining() < 8) { // type was later promoted to long return (long) tmp.getInt(); diff --git a/api/src/main/java/org/apache/iceberg/types/Type.java b/api/src/main/java/org/apache/iceberg/types/Type.java index 5062b54d10e1..571bf9a14e43 100644 --- a/api/src/main/java/org/apache/iceberg/types/Type.java +++ b/api/src/main/java/org/apache/iceberg/types/Type.java @@ -37,6 +37,7 @@ enum TypeID { DATE(Integer.class), TIME(Long.class), TIMESTAMP(Long.class), + TIMESTAMP_NANO(Long.class), STRING(CharSequence.class), UUID(java.util.UUID.class), FIXED(ByteBuffer.class), diff --git a/api/src/main/java/org/apache/iceberg/types/TypeUtil.java b/api/src/main/java/org/apache/iceberg/types/TypeUtil.java index 07d06dcc5a89..8a9184569aec 100644 --- a/api/src/main/java/org/apache/iceberg/types/TypeUtil.java +++ b/api/src/main/java/org/apache/iceberg/types/TypeUtil.java @@ -181,11 +181,36 @@ public static Map indexQuotedNameById( return indexer.byId(); } + /** + * Creates a mapping from lower-case field names to their corresponding field IDs. + * + *

This method iterates over the fields of the provided struct and maps each field's name + * (converted to lower-case) to its ID. If two fields have the same lower-case name, an + * `IllegalArgumentException` is thrown. + * + * @param struct the struct type whose fields are to be indexed + * @return a map where the keys are lower-case field names and the values are field IDs + * @throws IllegalArgumentException if two fields have the same lower-case name + */ public static Map indexByLowerCaseName(Types.StructType struct) { Map indexByLowerCaseName = Maps.newHashMap(); + + IndexByName indexer = new IndexByName(); + visit(struct, indexer); + Map byId = indexer.byId(); + indexByName(struct) .forEach( - (name, integer) -> indexByLowerCaseName.put(name.toLowerCase(Locale.ROOT), integer)); + (name, fieldId) -> { + String key = name.toLowerCase(Locale.ROOT); + Integer existingId = indexByLowerCaseName.put(key, fieldId); + Preconditions.checkArgument( + existingId == null || existingId.equals(fieldId), + "Cannot build lower case index: %s and %s collide", + byId.get(existingId), + byId.get(fieldId)); + indexByLowerCaseName.put(key, fieldId); + }); return indexByLowerCaseName; } @@ -496,6 +521,7 @@ private static int estimateSize(Type type) { case DOUBLE: case TIME: case TIMESTAMP: + case TIMESTAMP_NANO: // longs and doubles occupy 8 bytes // times and timestamps are internally represented as longs return 8; diff --git a/api/src/main/java/org/apache/iceberg/types/Types.java b/api/src/main/java/org/apache/iceberg/types/Types.java index ce6caa4721df..4bb1674f3be5 100644 --- a/api/src/main/java/org/apache/iceberg/types/Types.java +++ b/api/src/main/java/org/apache/iceberg/types/Types.java @@ -27,6 +27,7 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.relocated.com.google.common.base.Joiner; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -49,6 +50,8 @@ private Types() {} .put(TimeType.get().toString(), TimeType.get()) .put(TimestampType.withZone().toString(), TimestampType.withZone()) .put(TimestampType.withoutZone().toString(), TimestampType.withoutZone()) + .put(TimestampNanoType.withZone().toString(), TimestampNanoType.withZone()) + .put(TimestampNanoType.withoutZone().toString(), TimestampNanoType.withoutZone()) .put(StringType.get().toString(), StringType.get()) .put(UUIDType.get().toString(), UUIDType.get()) .put(BinaryType.get().toString(), BinaryType.get()) @@ -259,6 +262,59 @@ public int hashCode() { } } + public static class TimestampNanoType extends PrimitiveType { + private static final TimestampNanoType INSTANCE_WITH_ZONE = new TimestampNanoType(true); + private static final TimestampNanoType INSTANCE_WITHOUT_ZONE = new TimestampNanoType(false); + + public static TimestampNanoType withZone() { + return INSTANCE_WITH_ZONE; + } + + public static TimestampNanoType withoutZone() { + return INSTANCE_WITHOUT_ZONE; + } + + private final boolean adjustToUTC; + + private TimestampNanoType(boolean adjustToUTC) { + this.adjustToUTC = adjustToUTC; + } + + public boolean shouldAdjustToUTC() { + return adjustToUTC; + } + + @Override + public TypeID typeId() { + return TypeID.TIMESTAMP_NANO; + } + + @Override + public String toString() { + if (shouldAdjustToUTC()) { + return "timestamptz_ns"; + } else { + return "timestamp_ns"; + } + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } else if (!(other instanceof TimestampNanoType)) { + return false; + } + + return adjustToUTC == ((TimestampNanoType) other).adjustToUTC; + } + + @Override + public int hashCode() { + return Objects.hash(TimestampNanoType.class, adjustToUTC); + } + } + public static class StringType extends PrimitiveType { private static final StringType INSTANCE = new StringType(); @@ -317,7 +373,7 @@ public TypeID typeId() { @Override public String toString() { - return String.format("fixed[%d]", length); + return String.format(Locale.ROOT, "fixed[%d]", length); } @Override @@ -388,7 +444,7 @@ public TypeID typeId() { @Override public String toString() { - return String.format("decimal(%d, %d)", precision, scale); + return String.format(Locale.ROOT, "decimal(%d, %d)", precision, scale); } @Override @@ -414,27 +470,94 @@ public int hashCode() { public static class NestedField implements Serializable { public static NestedField optional(int id, String name, Type type) { - return new NestedField(true, id, name, type, null); + return new NestedField(true, id, name, type, null, null, null); } public static NestedField optional(int id, String name, Type type, String doc) { - return new NestedField(true, id, name, type, doc); + return new NestedField(true, id, name, type, doc, null, null); } public static NestedField required(int id, String name, Type type) { - return new NestedField(false, id, name, type, null); + return new NestedField(false, id, name, type, null, null, null); } public static NestedField required(int id, String name, Type type, String doc) { - return new NestedField(false, id, name, type, doc); + return new NestedField(false, id, name, type, doc, null, null); } public static NestedField of(int id, boolean isOptional, String name, Type type) { - return new NestedField(isOptional, id, name, type, null); + return new NestedField(isOptional, id, name, type, null, null, null); } public static NestedField of(int id, boolean isOptional, String name, Type type, String doc) { - return new NestedField(isOptional, id, name, type, doc); + return new NestedField(isOptional, id, name, type, doc, null, null); + } + + public static Builder from(NestedField field) { + return new Builder(field); + } + + public static Builder required(String name) { + return new Builder(false, name); + } + + public static Builder optional(String name) { + return new Builder(true, name); + } + + public static class Builder { + private final boolean isOptional; + private final String name; + private Integer id = null; + private Type type = null; + private String doc = null; + private Object initialDefault = null; + private Object writeDefault = null; + + private Builder(boolean isFieldOptional, String fieldName) { + isOptional = isFieldOptional; + name = fieldName; + } + + private Builder(NestedField toCopy) { + this.isOptional = toCopy.isOptional; + this.name = toCopy.name; + this.id = toCopy.id; + this.type = toCopy.type; + this.doc = toCopy.doc; + this.initialDefault = toCopy.initialDefault; + this.writeDefault = toCopy.writeDefault; + } + + public Builder withId(int fieldId) { + id = fieldId; + return this; + } + + public Builder ofType(Type fieldType) { + type = fieldType; + return this; + } + + public Builder withDoc(String fieldDoc) { + doc = fieldDoc; + return this; + } + + public Builder withInitialDefault(Object fieldInitialDefault) { + initialDefault = fieldInitialDefault; + return this; + } + + public Builder withWriteDefault(Object fieldWriteDefault) { + writeDefault = fieldWriteDefault; + return this; + } + + public NestedField build() { + // the constructor validates the fields + return new NestedField(isOptional, id, name, type, doc, initialDefault, writeDefault); + } } private final boolean isOptional; @@ -442,8 +565,17 @@ public static NestedField of(int id, boolean isOptional, String name, Type type, private final String name; private final Type type; private final String doc; - - private NestedField(boolean isOptional, int id, String name, Type type, String doc) { + private final Object initialDefault; + private final Object writeDefault; + + private NestedField( + boolean isOptional, + int id, + String name, + Type type, + String doc, + Object initialDefault, + Object writeDefault) { Preconditions.checkNotNull(name, "Name cannot be null"); Preconditions.checkNotNull(type, "Type cannot be null"); this.isOptional = isOptional; @@ -451,6 +583,19 @@ private NestedField(boolean isOptional, int id, String name, Type type, String d this.name = name; this.type = type; this.doc = doc; + this.initialDefault = castDefault(initialDefault, type); + this.writeDefault = castDefault(writeDefault, type); + } + + private static Object castDefault(Object defaultValue, Type type) { + if (type.isNestedType() && defaultValue != null) { + throw new IllegalArgumentException( + String.format("Invalid default value for %s: %s (must be null)", type, defaultValue)); + } else if (defaultValue != null) { + return Expressions.lit(defaultValue).to(type).value(); + } + + return null; } public boolean isOptional() { @@ -461,7 +606,7 @@ public NestedField asOptional() { if (isOptional) { return this; } - return new NestedField(true, id, name, type, doc); + return new NestedField(true, id, name, type, doc, initialDefault, writeDefault); } public boolean isRequired() { @@ -472,11 +617,15 @@ public NestedField asRequired() { if (!isOptional) { return this; } - return new NestedField(false, id, name, type, doc); + return new NestedField(false, id, name, type, doc, initialDefault, writeDefault); } + /** + * @deprecated will be removed in 2.0.0; use {@link Builder#withId(int)} instead + */ + @Deprecated public NestedField withFieldId(int newId) { - return new NestedField(isOptional, newId, name, type, doc); + return new NestedField(isOptional, newId, name, type, doc, initialDefault, writeDefault); } public int fieldId() { @@ -495,9 +644,18 @@ public String doc() { return doc; } + public Object initialDefault() { + return initialDefault; + } + + public Object writeDefault() { + return writeDefault; + } + @Override public String toString() { - return String.format("%d: %s: %s %s", id, name, isOptional ? "optional" : "required", type) + return String.format( + Locale.ROOT, "%d: %s: %s %s", id, name, isOptional ? "optional" : "required", type) + (doc != null ? " (" + doc + ")" : ""); } diff --git a/api/src/main/java/org/apache/iceberg/util/DataFileSet.java b/api/src/main/java/org/apache/iceberg/util/DataFileSet.java new file mode 100644 index 000000000000..27cbee088ad4 --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/util/DataFileSet.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.util; + +import java.util.Objects; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; + +public class DataFileSet extends WrapperSet { + private static final ThreadLocal WRAPPERS = + ThreadLocal.withInitial(() -> DataFileWrapper.wrap(null)); + + private DataFileSet() { + // needed for serialization/deserialization + } + + private DataFileSet(Iterable> wrappers) { + super(wrappers); + } + + public static DataFileSet create() { + return new DataFileSet(); + } + + public static DataFileSet of(Iterable iterable) { + return new DataFileSet( + Iterables.transform( + iterable, + obj -> { + Preconditions.checkNotNull(obj, "Invalid object: null"); + return DataFileWrapper.wrap(obj); + })); + } + + @Override + protected Wrapper wrapper() { + return WRAPPERS.get(); + } + + @Override + protected Wrapper wrap(DataFile dataFile) { + return DataFileWrapper.wrap(dataFile); + } + + @Override + protected Class elementClass() { + return DataFile.class; + } + + private static class DataFileWrapper implements Wrapper { + private DataFile file; + + private DataFileWrapper(DataFile file) { + this.file = file; + } + + private static DataFileWrapper wrap(DataFile dataFile) { + return new DataFileWrapper(dataFile); + } + + @Override + public DataFile get() { + return file; + } + + @Override + public Wrapper set(DataFile dataFile) { + this.file = dataFile; + return this; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof DataFileWrapper)) { + return false; + } + + DataFileWrapper that = (DataFileWrapper) o; + return Objects.equals(file.location(), that.file.location()); + } + + @Override + public int hashCode() { + return Objects.hashCode(file.location()); + } + + @Override + public String toString() { + return file.location(); + } + } +} diff --git a/api/src/main/java/org/apache/iceberg/util/DateTimeUtil.java b/api/src/main/java/org/apache/iceberg/util/DateTimeUtil.java index a2f5301f44a9..e26e7098cb22 100644 --- a/api/src/main/java/org/apache/iceberg/util/DateTimeUtil.java +++ b/api/src/main/java/org/apache/iceberg/util/DateTimeUtil.java @@ -27,6 +27,7 @@ import java.time.format.DateTimeFormatter; import java.time.format.DateTimeFormatterBuilder; import java.time.temporal.ChronoUnit; +import java.util.Locale; public class DateTimeUtil { private DateTimeUtil() {} @@ -35,6 +36,15 @@ private DateTimeUtil() {} public static final LocalDate EPOCH_DAY = EPOCH.toLocalDate(); public static final long MICROS_PER_MILLIS = 1000L; public static final long MICROS_PER_SECOND = 1_000_000L; + private static final long NANOS_PER_SECOND = 1_000_000_000L; + private static final long NANOS_PER_MICRO = 1_000L; + + private static final DateTimeFormatter FORMATTER = + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(DateTimeFormatter.ISO_LOCAL_DATE_TIME) + .appendOffset("+HH:MM:ss", "+00:00") + .toFormatter(Locale.ROOT); public static LocalDate dateFromDays(int daysFromEpoch) { return ChronoUnit.DAYS.addTo(EPOCH_DAY, daysFromEpoch); @@ -60,6 +70,10 @@ public static LocalDateTime timestampFromMicros(long microsFromEpoch) { return ChronoUnit.MICROS.addTo(EPOCH, microsFromEpoch).toLocalDateTime(); } + public static LocalDateTime timestampFromNanos(long nanosFromEpoch) { + return ChronoUnit.NANOS.addTo(EPOCH, nanosFromEpoch).toLocalDateTime(); + } + public static long microsFromInstant(Instant instant) { return ChronoUnit.MICROS.between(EPOCH, instant.atOffset(ZoneOffset.UTC)); } @@ -68,6 +82,10 @@ public static long microsFromTimestamp(LocalDateTime dateTime) { return ChronoUnit.MICROS.between(EPOCH, dateTime.atOffset(ZoneOffset.UTC)); } + public static long nanosFromTimestamp(LocalDateTime dateTime) { + return ChronoUnit.NANOS.between(EPOCH, dateTime.atOffset(ZoneOffset.UTC)); + } + public static long microsToMillis(long micros) { // When the timestamp is negative, i.e before 1970, we need to adjust the milliseconds portion. // Example - 1965-01-01 10:11:12.123456 is represented as (-157700927876544) in micro precision. @@ -75,6 +93,14 @@ public static long microsToMillis(long micros) { return Math.floorDiv(micros, MICROS_PER_MILLIS); } + public static long nanosToMicros(long nanos) { + return Math.floorDiv(nanos, NANOS_PER_MICRO); + } + + public static long microsToNanos(long micros) { + return Math.multiplyExact(micros, NANOS_PER_MICRO); + } + public static OffsetDateTime timestamptzFromMicros(long microsFromEpoch) { return ChronoUnit.MICROS.addTo(EPOCH, microsFromEpoch); } @@ -83,6 +109,10 @@ public static long microsFromTimestamptz(OffsetDateTime dateTime) { return ChronoUnit.MICROS.between(EPOCH, dateTime); } + public static long nanosFromTimestamptz(OffsetDateTime dateTime) { + return ChronoUnit.NANOS.between(EPOCH, dateTime); + } + public static String formatTimestampMillis(long millis) { return Instant.ofEpochMilli(millis).toString().replace("Z", "+00:00"); } @@ -97,13 +127,12 @@ public static String microsToIsoTime(long micros) { public static String microsToIsoTimestamptz(long micros) { LocalDateTime localDateTime = timestampFromMicros(micros); - DateTimeFormatter zeroOffsetFormatter = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(DateTimeFormatter.ISO_LOCAL_DATE_TIME) - .appendOffset("+HH:MM:ss", "+00:00") - .toFormatter(); - return localDateTime.atOffset(ZoneOffset.UTC).format(zeroOffsetFormatter); + return localDateTime.atOffset(ZoneOffset.UTC).format(FORMATTER); + } + + public static String nanosToIsoTimestamptz(long nanos) { + LocalDateTime localDateTime = timestampFromNanos(nanos); + return localDateTime.atOffset(ZoneOffset.UTC).format(FORMATTER); } public static String microsToIsoTimestamp(long micros) { @@ -111,6 +140,11 @@ public static String microsToIsoTimestamp(long micros) { return localDateTime.format(DateTimeFormatter.ISO_LOCAL_DATE_TIME); } + public static String nanosToIsoTimestamp(long nanos) { + LocalDateTime localDateTime = timestampFromNanos(nanos); + return localDateTime.format(DateTimeFormatter.ISO_LOCAL_DATE_TIME); + } + public static int isoDateToDays(String dateString) { return daysFromDate(LocalDate.parse(dateString, DateTimeFormatter.ISO_LOCAL_DATE)); } @@ -124,6 +158,11 @@ public static long isoTimestamptzToMicros(String timestampString) { OffsetDateTime.parse(timestampString, DateTimeFormatter.ISO_DATE_TIME)); } + public static long isoTimestamptzToNanos(CharSequence timestampString) { + return nanosFromTimestamptz( + OffsetDateTime.parse(timestampString, DateTimeFormatter.ISO_DATE_TIME)); + } + public static boolean isUTCTimestamptz(String timestampString) { OffsetDateTime offsetDateTime = OffsetDateTime.parse(timestampString, DateTimeFormatter.ISO_DATE_TIME); @@ -135,6 +174,11 @@ public static long isoTimestampToMicros(String timestampString) { LocalDateTime.parse(timestampString, DateTimeFormatter.ISO_LOCAL_DATE_TIME)); } + public static long isoTimestampToNanos(CharSequence timestampString) { + return nanosFromTimestamp( + LocalDateTime.parse(timestampString, DateTimeFormatter.ISO_LOCAL_DATE_TIME)); + } + public static int daysToYears(int days) { return convertDays(days, ChronoUnit.YEARS); } @@ -185,6 +229,36 @@ private static int convertMicros(long micros, ChronoUnit granularity) { } } + public static int nanosToYears(long nanos) { + return Math.toIntExact(convertNanos(nanos, ChronoUnit.YEARS)); + } + + public static int nanosToMonths(long nanos) { + return Math.toIntExact(convertNanos(nanos, ChronoUnit.MONTHS)); + } + + public static int nanosToDays(long nanos) { + return Math.toIntExact(convertNanos(nanos, ChronoUnit.DAYS)); + } + + public static int nanosToHours(long nanos) { + return Math.toIntExact(convertNanos(nanos, ChronoUnit.HOURS)); + } + + private static long convertNanos(long nanos, ChronoUnit granularity) { + if (nanos >= 0) { + long epochSecond = Math.floorDiv(nanos, NANOS_PER_SECOND); + long nanoAdjustment = Math.floorMod(nanos, NANOS_PER_SECOND); + return granularity.between(EPOCH, toOffsetDateTime(epochSecond, nanoAdjustment)); + } else { + // add 1 nano to the value to account for the case where there is exactly 1 unit between + // the timestamp and epoch because the result will always be decremented. + long epochSecond = Math.floorDiv(nanos, NANOS_PER_SECOND); + long nanoAdjustment = Math.floorMod(nanos + 1, NANOS_PER_SECOND); + return granularity.between(EPOCH, toOffsetDateTime(epochSecond, nanoAdjustment)) - 1; + } + } + private static OffsetDateTime toOffsetDateTime(long epochSecond, long nanoAdjustment) { return Instant.ofEpochSecond(epochSecond, nanoAdjustment).atOffset(ZoneOffset.UTC); } diff --git a/api/src/main/java/org/apache/iceberg/util/DeleteFileSet.java b/api/src/main/java/org/apache/iceberg/util/DeleteFileSet.java new file mode 100644 index 000000000000..bbe9824963fc --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/util/DeleteFileSet.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.util; + +import java.util.Objects; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; + +public class DeleteFileSet extends WrapperSet { + private static final ThreadLocal WRAPPERS = + ThreadLocal.withInitial(() -> DeleteFileWrapper.wrap(null)); + + private DeleteFileSet() { + // needed for serialization/deserialization + } + + private DeleteFileSet(Iterable> wrappers) { + super(wrappers); + } + + public static DeleteFileSet create() { + return new DeleteFileSet(); + } + + public static DeleteFileSet of(Iterable iterable) { + return new DeleteFileSet( + Iterables.transform( + iterable, + obj -> { + Preconditions.checkNotNull(obj, "Invalid object: null"); + return DeleteFileWrapper.wrap(obj); + })); + } + + @Override + protected Wrapper wrapper() { + return WRAPPERS.get(); + } + + @Override + protected Wrapper wrap(DeleteFile deleteFile) { + return DeleteFileWrapper.wrap(deleteFile); + } + + @Override + protected Class elementClass() { + return DeleteFile.class; + } + + private static class DeleteFileWrapper implements Wrapper { + private DeleteFile file; + + private DeleteFileWrapper(DeleteFile file) { + this.file = file; + } + + private static DeleteFileWrapper wrap(DeleteFile deleteFile) { + return new DeleteFileWrapper(deleteFile); + } + + @Override + public DeleteFile get() { + return file; + } + + @Override + public Wrapper set(DeleteFile deleteFile) { + this.file = deleteFile; + return this; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof DeleteFileWrapper)) { + return false; + } + + DeleteFileWrapper that = (DeleteFileWrapper) o; + // this needs to be updated once deletion vector support is added + return Objects.equals(file.location(), that.file.location()); + } + + @Override + public int hashCode() { + return Objects.hashCode(file.location()); + } + + @Override + public String toString() { + return file.location(); + } + } +} diff --git a/api/src/main/java/org/apache/iceberg/util/UnicodeUtil.java b/api/src/main/java/org/apache/iceberg/util/UnicodeUtil.java index 4dd2afa123ac..a1bb3f497196 100644 --- a/api/src/main/java/org/apache/iceberg/util/UnicodeUtil.java +++ b/api/src/main/java/org/apache/iceberg/util/UnicodeUtil.java @@ -82,9 +82,9 @@ public static Literal truncateStringMax(Literal inpu for (int i = length - 1; i >= 0; i--) { // Get the offset in the truncated string buffer where the number of unicode characters = i int offsetByCodePoint = truncatedStringBuilder.offsetByCodePoints(0, i); - int nextCodePoint = truncatedStringBuilder.codePointAt(offsetByCodePoint) + 1; + int nextCodePoint = incrementCodePoint(truncatedStringBuilder.codePointAt(offsetByCodePoint)); // No overflow - if (nextCodePoint != 0 && Character.isValidCodePoint(nextCodePoint)) { + if (nextCodePoint != 0) { truncatedStringBuilder.setLength(offsetByCodePoint); // Append next code point to the truncated substring truncatedStringBuilder.appendCodePoint(nextCodePoint); @@ -93,4 +93,24 @@ public static Literal truncateStringMax(Literal inpu } return null; // Cannot find a valid upper bound } + + private static int incrementCodePoint(int codePoint) { + // surrogate code points are not Unicode scalar values, + // any UTF-8 byte sequence that would otherwise map to code points U+D800..U+DFFF is ill-formed. + // see https://www.unicode.org/versions/Unicode16.0.0/core-spec/chapter-3/#G27288 + Preconditions.checkArgument( + codePoint < Character.MIN_SURROGATE || codePoint > Character.MAX_SURROGATE, + "invalid code point: %s", + codePoint); + + if (codePoint == Character.MIN_SURROGATE - 1) { + // increment to the next Unicode scalar value + return Character.MAX_SURROGATE + 1; + } else if (codePoint == Character.MAX_CODE_POINT) { + // overflow + return 0; + } else { + return codePoint + 1; + } + } } diff --git a/api/src/main/java/org/apache/iceberg/util/WrapperSet.java b/api/src/main/java/org/apache/iceberg/util/WrapperSet.java new file mode 100644 index 000000000000..e589f435e158 --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/util/WrapperSet.java @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.util; + +import java.io.Serializable; +import java.util.Collection; +import java.util.Iterator; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Iterators; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.relocated.com.google.common.collect.Streams; + +/** + * A custom set for a {@link Wrapper} of the given type that maintains insertion order and does not + * allow null elements. + * + * @param The type to wrap in a {@link Wrapper} instance. + */ +abstract class WrapperSet implements Set, Serializable { + private final Set> set = Sets.newLinkedHashSet(); + + protected WrapperSet(Iterable> wrappers) { + wrappers.forEach(set::add); + } + + protected WrapperSet() {} + + protected abstract Wrapper wrapper(); + + protected abstract Wrapper wrap(T file); + + protected abstract Class elementClass(); + + protected interface Wrapper extends Serializable { + T get(); + + Wrapper set(T object); + } + + @Override + public int size() { + return set.size(); + } + + @Override + public boolean isEmpty() { + return set.isEmpty(); + } + + @Override + public boolean contains(Object obj) { + Preconditions.checkNotNull(obj, "Invalid object: null"); + Wrapper wrapper = wrapper(); + boolean result = set.contains(wrapper.set(elementClass().cast(obj))); + wrapper.set(null); // don't hold a reference to the value + return result; + } + + @Override + public Iterator iterator() { + return Iterators.transform(set.iterator(), Wrapper::get); + } + + @Override + public Object[] toArray() { + return Lists.newArrayList(iterator()).toArray(); + } + + @Override + public X[] toArray(X[] destArray) { + return Lists.newArrayList(iterator()).toArray(destArray); + } + + @Override + public boolean add(T obj) { + Preconditions.checkNotNull(obj, "Invalid object: null"); + return set.add(wrap(obj)); + } + + @Override + public boolean remove(Object obj) { + Preconditions.checkNotNull(obj, "Invalid object: null"); + Wrapper wrapper = wrapper(); + boolean result = set.remove(wrapper.set(elementClass().cast(obj))); + wrapper.set(null); // don't hold a reference to the value + return result; + } + + @Override + public boolean containsAll(Collection collection) { + Preconditions.checkNotNull(collection, "Invalid collection: null"); + return Iterables.all(collection, this::contains); + } + + @Override + public boolean addAll(Collection collection) { + Preconditions.checkNotNull(collection, "Invalid collection: null"); + return collection.stream().filter(this::add).count() != 0; + } + + @Override + public boolean retainAll(Collection collection) { + Preconditions.checkNotNull(collection, "Invalid collection: null"); + Set> toRetain = + collection.stream() + .map(obj -> Preconditions.checkNotNull(obj, "Invalid object: null")) + .map(elementClass()::cast) + .map(this::wrap) + .collect(Collectors.toSet()); + + return Iterables.retainAll(set, toRetain); + } + + @Override + public boolean removeAll(Collection collection) { + Preconditions.checkNotNull(collection, "Invalid collection: null"); + return collection.stream().filter(this::remove).count() != 0; + } + + @Override + public void clear() { + set.clear(); + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } else if (!(other instanceof Set)) { + return false; + } + + Set that = (Set) other; + + if (size() != that.size()) { + return false; + } + + try { + return containsAll(that); + } catch (ClassCastException | NullPointerException unused) { + return false; + } + } + + @Override + public int hashCode() { + return set.stream().mapToInt(Object::hashCode).sum(); + } + + @Override + public String toString() { + return Streams.stream(iterator()) + .map(Object::toString) + .collect(Collectors.joining(", ", "[", "]")); + } +} diff --git a/api/src/test/java/org/apache/iceberg/ParameterizedTestExtension.java b/api/src/test/java/org/apache/iceberg/ParameterizedTestExtension.java index 59652bab9851..2f625092ff7c 100644 --- a/api/src/test/java/org/apache/iceberg/ParameterizedTestExtension.java +++ b/api/src/test/java/org/apache/iceberg/ParameterizedTestExtension.java @@ -73,13 +73,10 @@ public Stream provideTestTemplateInvocationContex // Search method annotated with @Parameters final List parameterProviders = AnnotationSupport.findAnnotatedMethods( - context.getRequiredTestClass(), Parameters.class, HierarchyTraversalMode.TOP_DOWN); + context.getRequiredTestClass(), Parameters.class, HierarchyTraversalMode.BOTTOM_UP); if (parameterProviders.isEmpty()) { throw new IllegalStateException("Cannot find any parameter provider"); } - if (parameterProviders.size() > 1) { - throw new IllegalStateException("Multiple parameter providers are found"); - } Method parameterProvider = parameterProviders.get(0); // Get potential test name @@ -225,7 +222,11 @@ private Stream createContextForParameters( Stream parameterValueStream, String testNameTemplate, ExtensionContext context) { // Search fields annotated by @Parameter final List parameterFields = - AnnotationSupport.findAnnotatedFields(context.getRequiredTestClass(), Parameter.class); + AnnotationSupport.findAnnotatedFields( + context.getRequiredTestClass(), + Parameter.class, + field -> true, + HierarchyTraversalMode.BOTTOM_UP); // Use constructor parameter style if (parameterFields.isEmpty()) { diff --git a/api/src/test/java/org/apache/iceberg/PartitionSpecTestBase.java b/api/src/test/java/org/apache/iceberg/PartitionSpecTestBase.java index 5e4ca1fb11be..11f2cb353880 100644 --- a/api/src/test/java/org/apache/iceberg/PartitionSpecTestBase.java +++ b/api/src/test/java/org/apache/iceberg/PartitionSpecTestBase.java @@ -34,7 +34,8 @@ public class PartitionSpecTestBase { Types.NestedField.required(7, "s", Types.StringType.get()), Types.NestedField.required(8, "u", Types.UUIDType.get()), Types.NestedField.required(9, "f", Types.FixedType.ofLength(3)), - Types.NestedField.required(10, "b", Types.BinaryType.get())); + Types.NestedField.required(10, "b", Types.BinaryType.get()), + Types.NestedField.required(11, "tsn", Types.TimestampNanoType.withoutZone())); // a spec with all of the allowed transform/type pairs public static final PartitionSpec[] SPECS = @@ -49,6 +50,7 @@ public class PartitionSpecTestBase { PartitionSpec.builderFor(SCHEMA).identity("u").build(), PartitionSpec.builderFor(SCHEMA).identity("f").build(), PartitionSpec.builderFor(SCHEMA).identity("b").build(), + PartitionSpec.builderFor(SCHEMA).identity("tsn").build(), PartitionSpec.builderFor(SCHEMA).bucket("i", 128).build(), PartitionSpec.builderFor(SCHEMA).bucket("l", 128).build(), PartitionSpec.builderFor(SCHEMA).bucket("d", 128).build(), @@ -59,6 +61,7 @@ public class PartitionSpecTestBase { PartitionSpec.builderFor(SCHEMA).bucket("u", 128).build(), PartitionSpec.builderFor(SCHEMA).bucket("f", 128).build(), PartitionSpec.builderFor(SCHEMA).bucket("b", 128).build(), + PartitionSpec.builderFor(SCHEMA).bucket("tsn", 128).build(), PartitionSpec.builderFor(SCHEMA).year("d").build(), PartitionSpec.builderFor(SCHEMA).month("d").build(), PartitionSpec.builderFor(SCHEMA).day("d").build(), @@ -66,6 +69,10 @@ public class PartitionSpecTestBase { PartitionSpec.builderFor(SCHEMA).month("ts").build(), PartitionSpec.builderFor(SCHEMA).day("ts").build(), PartitionSpec.builderFor(SCHEMA).hour("ts").build(), + PartitionSpec.builderFor(SCHEMA).year("tsn").build(), + PartitionSpec.builderFor(SCHEMA).month("tsn").build(), + PartitionSpec.builderFor(SCHEMA).day("tsn").build(), + PartitionSpec.builderFor(SCHEMA).hour("tsn").build(), PartitionSpec.builderFor(SCHEMA).truncate("i", 10).build(), PartitionSpec.builderFor(SCHEMA).truncate("l", 10).build(), PartitionSpec.builderFor(SCHEMA).truncate("dec", 10).build(), diff --git a/api/src/test/java/org/apache/iceberg/TestAccessors.java b/api/src/test/java/org/apache/iceberg/TestAccessors.java index 332556e474c7..7b4feb845f12 100644 --- a/api/src/test/java/org/apache/iceberg/TestAccessors.java +++ b/api/src/test/java/org/apache/iceberg/TestAccessors.java @@ -180,6 +180,8 @@ public void testTime() { public void testTimestamp() { assertAccessorReturns(Types.TimestampType.withoutZone(), 123L); assertAccessorReturns(Types.TimestampType.withZone(), 123L); + assertAccessorReturns(Types.TimestampNanoType.withoutZone(), 123L); + assertAccessorReturns(Types.TimestampNanoType.withZone(), 123L); } @Test diff --git a/api/src/test/java/org/apache/iceberg/TestHelpers.java b/api/src/test/java/org/apache/iceberg/TestHelpers.java index 329a61fb44b3..ca3b1a908ac6 100644 --- a/api/src/test/java/org/apache/iceberg/TestHelpers.java +++ b/api/src/test/java/org/apache/iceberg/TestHelpers.java @@ -222,6 +222,7 @@ public static T deserialize(final InputStream inputStream) return obj; } } + /** * Serializes an {@link Object} to a byte array for storage/serialization. * diff --git a/api/src/test/java/org/apache/iceberg/TestPartitionPaths.java b/api/src/test/java/org/apache/iceberg/TestPartitionPaths.java index 5455415da015..31a6c486bf6e 100644 --- a/api/src/test/java/org/apache/iceberg/TestPartitionPaths.java +++ b/api/src/test/java/org/apache/iceberg/TestPartitionPaths.java @@ -54,6 +54,44 @@ public void testPartitionPath() { .isEqualTo("ts_hour=2017-12-01-10/id_bucket=" + idBucket); } + @Test + public void testPartitionPathWithNanoseconds() { + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).hour("ts").bucket("id", 10).build(); + + Transform hour = Transforms.hour(); + Transform bucket = Transforms.bucket(10); + + Literal ts = + Literal.of("2017-12-01T10:12:55.038194789").to(Types.TimestampNanoType.withoutZone()); + Object tsHour = hour.bind(Types.TimestampNanoType.withoutZone()).apply(ts.value()); + Object idBucket = bucket.bind(Types.IntegerType.get()).apply(1); + + Row partition = Row.of(tsHour, idBucket); + + assertThat(spec.partitionToPath(partition)) + .as("Should produce expected partition key") + .isEqualTo("ts_hour=2017-12-01-10/id_bucket=" + idBucket); + } + + @Test + public void testPartitionPathWithNanosecondsTz() { + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).hour("ts").bucket("id", 10).build(); + + Transform hour = Transforms.hour(); + Transform bucket = Transforms.bucket(10); + + Literal ts = + Literal.of("2017-12-01T10:12:55.038194789-08:00").to(Types.TimestampNanoType.withZone()); + Object tsTzHour = hour.bind(Types.TimestampNanoType.withZone()).apply(ts.value()); + Object idBucket = bucket.bind(Types.IntegerType.get()).apply(1); + + Row partition = Row.of(tsTzHour, idBucket); + + assertThat(spec.partitionToPath(partition)) + .as("Should produce expected partition key") + .isEqualTo("ts_hour=2017-12-01-18/id_bucket=" + idBucket); + } + @Test public void testEscapedStrings() { PartitionSpec spec = diff --git a/api/src/test/java/org/apache/iceberg/TestSchemaCaseSensitivity.java b/api/src/test/java/org/apache/iceberg/TestSchemaCaseSensitivity.java new file mode 100644 index 000000000000..bdb73374bf21 --- /dev/null +++ b/api/src/test/java/org/apache/iceberg/TestSchemaCaseSensitivity.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatIllegalArgumentException; + +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestSchemaCaseSensitivity { + + @Test + public void testCaseInsensitiveFieldCollision() { + Schema schema = + new Schema( + required(1, "id", Types.LongType.get()), + required(2, "data", Types.StringType.get()), + required(3, "DATA", Types.StringType.get())); + assertThatIllegalArgumentException() + .isThrownBy(() -> schema.caseInsensitiveFindField("DATA")) + .withMessage("Cannot build lower case index: data and DATA collide"); + } + + @Test + public void testCaseSensitiveFindField() { + Schema schema = + new Schema( + required(1, "id", Types.LongType.get()), + required(2, "data", Types.StringType.get()), + required(3, "DATA", Types.StringType.get())); + + Types.NestedField actual1 = schema.findField("data"); + assertThat(actual1).isEqualTo(Types.NestedField.required(2, "data", Types.StringType.get())); + Types.NestedField actual2 = schema.findField("DATA"); + assertThat(actual2).isEqualTo(Types.NestedField.required(3, "DATA", Types.StringType.get())); + } + + @Test + public void testCaseInsensitiveField() { + Schema schema = + new Schema( + required(1, "id", Types.LongType.get()), required(2, "data", Types.StringType.get())); + + Types.NestedField actual1 = schema.caseInsensitiveFindField("DATA"); + assertThat(actual1).isEqualTo(Types.NestedField.required(2, "data", Types.StringType.get())); + } +} diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestExpressionUtil.java b/api/src/test/java/org/apache/iceberg/expressions/TestExpressionUtil.java index 894989f38e7d..10d3b6d0adfa 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestExpressionUtil.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestExpressionUtil.java @@ -42,11 +42,12 @@ public class TestExpressionUtil { Types.NestedField.required(2, "val", Types.IntegerType.get()), Types.NestedField.required(3, "val2", Types.IntegerType.get()), Types.NestedField.required(4, "ts", Types.TimestampType.withoutZone()), - Types.NestedField.required(5, "date", Types.DateType.get()), - Types.NestedField.required(6, "time", Types.DateType.get()), - Types.NestedField.optional(7, "data", Types.StringType.get()), - Types.NestedField.optional(8, "measurement", Types.DoubleType.get()), - Types.NestedField.optional(9, "test", Types.IntegerType.get())); + Types.NestedField.required(5, "tsns", Types.TimestampNanoType.withoutZone()), + Types.NestedField.required(6, "date", Types.DateType.get()), + Types.NestedField.required(7, "time", Types.DateType.get()), + Types.NestedField.optional(8, "data", Types.StringType.get()), + Types.NestedField.optional(9, "measurement", Types.DoubleType.get()), + Types.NestedField.optional(10, "test", Types.IntegerType.get())); private static final Types.StructType STRUCT = SCHEMA.asStruct(); @@ -460,7 +461,9 @@ public void testSanitizeTimestamp() { "2022-04-29T23:49:51", "2022-04-29T23:49:51.123456", "2022-04-29T23:49:51-07:00", - "2022-04-29T23:49:51.123456+01:00")) { + "2022-04-29T23:49:51.123456+01:00", + "2022-04-29T23:49:51.123456789", + "2022-04-29T23:49:51.123456789+01:00")) { assertEquals( Expressions.equal("test", "(timestamp)"), ExpressionUtil.sanitize(Expressions.equal("test", timestamp))); @@ -496,6 +499,13 @@ public void testSanitizeTimestampAboutNow() { Expression.Operation.EQ, "test", Literal.of(nowLocal).to(Types.TimestampType.withoutZone())))); + assertEquals( + Expressions.equal("test", "(timestamp-about-now)"), + ExpressionUtil.sanitize( + Expressions.predicate( + Expression.Operation.EQ, + "test", + Literal.of(nowLocal).to(Types.TimestampNanoType.withoutZone())))); assertThat(ExpressionUtil.toSanitizedString(Expressions.equal("test", nowLocal))) .as("Sanitized string should be identical except for descriptive literal") @@ -522,6 +532,13 @@ public void testSanitizeTimestampPast() { Expression.Operation.EQ, "test", Literal.of(ninetyMinutesAgoLocal).to(Types.TimestampType.withoutZone())))); + assertEquals( + Expressions.equal("test", "(timestamp-1-hours-ago)"), + ExpressionUtil.sanitize( + Expressions.predicate( + Expression.Operation.EQ, + "test", + Literal.of(ninetyMinutesAgoLocal).to(Types.TimestampNanoType.withoutZone())))); assertThat(ExpressionUtil.toSanitizedString(Expressions.equal("test", ninetyMinutesAgoLocal))) .as("Sanitized string should be identical except for descriptive literal") @@ -548,6 +565,13 @@ public void testSanitizeTimestampLastWeek() { Expression.Operation.EQ, "test", Literal.of(lastWeekLocal).to(Types.TimestampType.withoutZone())))); + assertEquals( + Expressions.equal("test", "(timestamp-7-days-ago)"), + ExpressionUtil.sanitize( + Expressions.predicate( + Expression.Operation.EQ, + "test", + Literal.of(lastWeekLocal).to(Types.TimestampNanoType.withoutZone())))); assertThat(ExpressionUtil.toSanitizedString(Expressions.equal("test", lastWeekLocal))) .as("Sanitized string should be identical except for descriptive literal") @@ -574,6 +598,13 @@ public void testSanitizeTimestampFuture() { Expression.Operation.EQ, "test", Literal.of(ninetyMinutesFromNowLocal).to(Types.TimestampType.withoutZone())))); + assertEquals( + Expressions.equal("test", "(timestamp-1-hours-from-now)"), + ExpressionUtil.sanitize( + Expressions.predicate( + Expression.Operation.EQ, + "test", + Literal.of(ninetyMinutesFromNowLocal).to(Types.TimestampNanoType.withoutZone())))); assertThat( ExpressionUtil.toSanitizedString(Expressions.equal("test", ninetyMinutesFromNowLocal))) @@ -597,6 +628,13 @@ public void testSanitizeTimestamptzAboutNow() { Expression.Operation.EQ, "test", Literal.of(nowUtc).to(Types.TimestampType.withZone())))); + assertEquals( + Expressions.equal("test", "(timestamp-about-now)"), + ExpressionUtil.sanitize( + Expressions.predicate( + Expression.Operation.EQ, + "test", + Literal.of(nowUtc).to(Types.TimestampNanoType.withZone())))); assertThat(ExpressionUtil.toSanitizedString(Expressions.equal("test", nowUtc))) .as("Sanitized string should be identical except for descriptive literal") @@ -618,6 +656,13 @@ public void testSanitizeTimestamptzPast() { Expression.Operation.EQ, "test", Literal.of(ninetyMinutesAgoUtc).to(Types.TimestampType.withZone())))); + assertEquals( + Expressions.equal("test", "(timestamp-1-hours-ago)"), + ExpressionUtil.sanitize( + Expressions.predicate( + Expression.Operation.EQ, + "test", + Literal.of(ninetyMinutesAgoUtc).to(Types.TimestampNanoType.withZone())))); assertThat(ExpressionUtil.toSanitizedString(Expressions.equal("test", ninetyMinutesAgoUtc))) .as("Sanitized string should be identical except for descriptive literal") @@ -639,6 +684,13 @@ public void testSanitizeTimestamptzLastWeek() { Expression.Operation.EQ, "test", Literal.of(lastWeekUtc).to(Types.TimestampType.withZone())))); + assertEquals( + Expressions.equal("test", "(timestamp-7-days-ago)"), + ExpressionUtil.sanitize( + Expressions.predicate( + Expression.Operation.EQ, + "test", + Literal.of(lastWeekUtc).to(Types.TimestampNanoType.withZone())))); assertThat(ExpressionUtil.toSanitizedString(Expressions.equal("test", lastWeekUtc))) .as("Sanitized string should be identical except for descriptive literal") @@ -660,6 +712,13 @@ public void testSanitizeTimestamptzFuture() { Expression.Operation.EQ, "test", Literal.of(ninetyMinutesFromNowUtc).to(Types.TimestampType.withZone())))); + assertEquals( + Expressions.equal("test", "(timestamp-1-hours-from-now)"), + ExpressionUtil.sanitize( + Expressions.predicate( + Expression.Operation.EQ, + "test", + Literal.of(ninetyMinutesFromNowUtc).to(Types.TimestampNanoType.withZone())))); assertThat(ExpressionUtil.toSanitizedString(Expressions.equal("test", ninetyMinutesFromNowUtc))) .as("Sanitized string should be identical except for descriptive literal") diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestLiteralSerialization.java b/api/src/test/java/org/apache/iceberg/expressions/TestLiteralSerialization.java index d5aa251ffb50..24fc458b37b4 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestLiteralSerialization.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestLiteralSerialization.java @@ -38,8 +38,10 @@ public void testLiterals() throws Exception { Literal.of(8.75D), Literal.of("2017-11-29").to(Types.DateType.get()), Literal.of("11:30:07").to(Types.TimeType.get()), - Literal.of("2017-11-29T11:30:07.123").to(Types.TimestampType.withoutZone()), - Literal.of("2017-11-29T11:30:07.123+01:00").to(Types.TimestampType.withZone()), + Literal.of("2017-11-29T11:30:07.123456").to(Types.TimestampType.withoutZone()), + Literal.of("2017-11-29T11:30:07.123456+01:00").to(Types.TimestampType.withZone()), + Literal.of("2017-11-29T11:30:07.123456789").to(Types.TimestampNanoType.withoutZone()), + Literal.of("2017-11-29T11:30:07.123456789+01:00").to(Types.TimestampNanoType.withZone()), Literal.of("abc"), Literal.of(UUID.randomUUID()), Literal.of(new byte[] {1, 2, 3}).to(Types.FixedType.ofLength(3)), diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestMiscLiteralConversions.java b/api/src/test/java/org/apache/iceberg/expressions/TestMiscLiteralConversions.java index f8d2cd49d969..e2611ddb281f 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestMiscLiteralConversions.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestMiscLiteralConversions.java @@ -42,7 +42,9 @@ public void testIdentityConversions() { Pair.of(Literal.of("34.55"), Types.DecimalType.of(9, 2)), Pair.of(Literal.of("2017-08-18"), Types.DateType.get()), Pair.of(Literal.of("14:21:01.919"), Types.TimeType.get()), - Pair.of(Literal.of("2017-08-18T14:21:01.919"), Types.TimestampType.withoutZone()), + Pair.of(Literal.of("2017-08-18T14:21:01.919432"), Types.TimestampType.withoutZone()), + Pair.of( + Literal.of("2017-08-18T14:21:01.919432755"), Types.TimestampNanoType.withoutZone()), Pair.of(Literal.of("abc"), Types.StringType.get()), Pair.of(Literal.of(UUID.randomUUID()), Types.UUIDType.get()), Pair.of(Literal.of(new byte[] {0, 1, 2}), Types.FixedType.ofLength(3)), @@ -62,6 +64,22 @@ public void testIdentityConversions() { } } + @Test + public void testTimestampWithMicrosecondsToDate() { + final Literal micros = + Literal.of("2017-08-18T14:21:01.919432755").to(Types.TimestampType.withoutZone()); + final Literal dateOfNanos = micros.to(Types.DateType.get()); + assertThat(dateOfNanos).isEqualTo(Literal.of("2017-08-18").to(Types.DateType.get())); + } + + @Test + public void testTimestampWithNanosecondsToDate() { + final Literal nanos = + Literal.of("2017-08-18T14:21:01.919432755").to(Types.TimestampNanoType.withoutZone()); + final Literal dateOfNanos = nanos.to(Types.DateType.get()); + assertThat(dateOfNanos).isEqualTo(Literal.of("2017-08-18").to(Types.DateType.get())); + } + @Test public void testBinaryToFixed() { Literal lit = Literal.of(ByteBuffer.wrap(new byte[] {0, 1, 2})); @@ -101,6 +119,8 @@ public void testInvalidBooleanConversions() { Types.TimeType.get(), Types.TimestampType.withZone(), Types.TimestampType.withoutZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.DecimalType.of(9, 2), Types.StringType.get(), Types.UUIDType.get(), @@ -116,6 +136,8 @@ public void testInvalidIntegerConversions() { Types.TimeType.get(), Types.TimestampType.withZone(), Types.TimestampType.withoutZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.StringType.get(), Types.UUIDType.get(), Types.FixedType.ofLength(1), @@ -144,6 +166,8 @@ public void testInvalidFloatConversions() { Types.TimeType.get(), Types.TimestampType.withZone(), Types.TimestampType.withoutZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.StringType.get(), Types.UUIDType.get(), Types.FixedType.ofLength(1), @@ -161,6 +185,8 @@ public void testInvalidDoubleConversions() { Types.TimeType.get(), Types.TimestampType.withZone(), Types.TimestampType.withoutZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.StringType.get(), Types.UUIDType.get(), Types.FixedType.ofLength(1), @@ -179,6 +205,8 @@ public void testInvalidDateConversions() { Types.TimeType.get(), Types.TimestampType.withZone(), Types.TimestampType.withoutZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.DecimalType.of(9, 4), Types.StringType.get(), Types.UUIDType.get(), @@ -198,6 +226,8 @@ public void testInvalidTimeConversions() { Types.DateType.get(), Types.TimestampType.withZone(), Types.TimestampType.withoutZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.DecimalType.of(9, 4), Types.StringType.get(), Types.UUIDType.get(), @@ -206,9 +236,26 @@ public void testInvalidTimeConversions() { } @Test - public void testInvalidTimestampConversions() { + public void testInvalidTimestampMicrosConversions() { testInvalidConversions( - Literal.of("2017-08-18T14:21:01.919").to(Types.TimestampType.withoutZone()), + Literal.of("2017-08-18T14:21:01.919123").to(Types.TimestampType.withoutZone()), + Types.BooleanType.get(), + Types.IntegerType.get(), + Types.LongType.get(), + Types.FloatType.get(), + Types.DoubleType.get(), + Types.TimeType.get(), + Types.DecimalType.of(9, 4), + Types.StringType.get(), + Types.UUIDType.get(), + Types.FixedType.ofLength(1), + Types.BinaryType.get()); + } + + @Test + public void testInvalidTimestampNanosConversions() { + testInvalidConversions( + Literal.of("2017-08-18T14:21:01.919123456").to(Types.TimestampNanoType.withoutZone()), Types.BooleanType.get(), Types.IntegerType.get(), Types.LongType.get(), @@ -233,8 +280,10 @@ public void testInvalidDecimalConversions() { Types.DoubleType.get(), Types.DateType.get(), Types.TimeType.get(), - Types.TimestampType.withZone(), Types.TimestampType.withoutZone(), + Types.TimestampType.withZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.StringType.get(), Types.UUIDType.get(), Types.FixedType.ofLength(1), @@ -267,8 +316,10 @@ public void testInvalidUUIDConversions() { Types.DoubleType.get(), Types.DateType.get(), Types.TimeType.get(), - Types.TimestampType.withZone(), Types.TimestampType.withoutZone(), + Types.TimestampType.withZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.DecimalType.of(9, 2), Types.StringType.get(), Types.FixedType.ofLength(1), @@ -286,8 +337,10 @@ public void testInvalidFixedConversions() { Types.DoubleType.get(), Types.DateType.get(), Types.TimeType.get(), - Types.TimestampType.withZone(), Types.TimestampType.withoutZone(), + Types.TimestampType.withZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.DecimalType.of(9, 2), Types.StringType.get(), Types.UUIDType.get(), @@ -307,6 +360,8 @@ public void testInvalidBinaryConversions() { Types.TimeType.get(), Types.TimestampType.withZone(), Types.TimestampType.withoutZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.DecimalType.of(9, 2), Types.StringType.get(), Types.UUIDType.get(), diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestStrictMetricsEvaluator.java b/api/src/test/java/org/apache/iceberg/expressions/TestStrictMetricsEvaluator.java index a23afb15dde5..f34cd730df77 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestStrictMetricsEvaluator.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestStrictMetricsEvaluator.java @@ -66,7 +66,14 @@ public class TestStrictMetricsEvaluator { optional(11, "all_nulls_double", Types.DoubleType.get()), optional(12, "all_nans_v1_stats", Types.FloatType.get()), optional(13, "nan_and_null_only", Types.DoubleType.get()), - optional(14, "no_nan_stats", Types.DoubleType.get())); + optional(14, "no_nan_stats", Types.DoubleType.get()), + optional( + 15, + "struct", + Types.StructType.of( + Types.NestedField.optional(16, "nested_col_no_stats", Types.IntegerType.get()), + Types.NestedField.optional( + 17, "nested_col_with_stats", Types.IntegerType.get())))); private static final int INT_MIN_VALUE = 30; private static final int INT_MAX_VALUE = 79; @@ -88,6 +95,7 @@ public class TestStrictMetricsEvaluator { .put(12, 50L) .put(13, 50L) .put(14, 50L) + .put(17, 50L) .buildOrThrow(), // null value counts ImmutableMap.builder() @@ -97,6 +105,7 @@ public class TestStrictMetricsEvaluator { .put(11, 50L) .put(12, 0L) .put(13, 1L) + .put(17, 0L) .buildOrThrow(), // nan value counts ImmutableMap.of( @@ -108,13 +117,15 @@ public class TestStrictMetricsEvaluator { 1, toByteBuffer(IntegerType.get(), INT_MIN_VALUE), 7, toByteBuffer(IntegerType.get(), 5), 12, toByteBuffer(Types.FloatType.get(), Float.NaN), - 13, toByteBuffer(Types.DoubleType.get(), Double.NaN)), + 13, toByteBuffer(Types.DoubleType.get(), Double.NaN), + 17, toByteBuffer(Types.IntegerType.get(), INT_MIN_VALUE)), // upper bounds ImmutableMap.of( 1, toByteBuffer(IntegerType.get(), INT_MAX_VALUE), 7, toByteBuffer(IntegerType.get(), 5), 12, toByteBuffer(Types.FloatType.get(), Float.NaN), - 13, toByteBuffer(Types.DoubleType.get(), Double.NaN))); + 13, toByteBuffer(Types.DoubleType.get(), Double.NaN), + 17, toByteBuffer(IntegerType.get(), INT_MAX_VALUE))); private static final DataFile FILE_2 = new TestDataFile( @@ -627,4 +638,50 @@ public void testIntegerNotIn() { shouldRead = new StrictMetricsEvaluator(SCHEMA, notIn("no_nulls", "abc", "def")).eval(FILE); assertThat(shouldRead).as("Should not match: no_nulls field does not have bounds").isFalse(); } + + @Test + public void testEvaluateOnNestedColumnWithoutStats() { + boolean shouldRead = + new StrictMetricsEvaluator( + SCHEMA, greaterThanOrEqual("struct.nested_col_no_stats", INT_MIN_VALUE)) + .eval(FILE); + assertThat(shouldRead).as("greaterThanOrEqual nested column should not match").isFalse(); + + shouldRead = + new StrictMetricsEvaluator( + SCHEMA, lessThanOrEqual("struct.nested_col_no_stats", INT_MAX_VALUE)) + .eval(FILE); + assertThat(shouldRead).as("lessThanOrEqual nested column should not match").isFalse(); + + shouldRead = + new StrictMetricsEvaluator(SCHEMA, isNull("struct.nested_col_no_stats")).eval(FILE); + assertThat(shouldRead).as("isNull nested column should not match").isFalse(); + + shouldRead = + new StrictMetricsEvaluator(SCHEMA, notNull("struct.nested_col_no_stats")).eval(FILE); + assertThat(shouldRead).as("notNull nested column should not match").isFalse(); + } + + @Test + public void testEvaluateOnNestedColumnWithStats() { + boolean shouldRead = + new StrictMetricsEvaluator( + SCHEMA, greaterThanOrEqual("struct.nested_col_with_stats", INT_MIN_VALUE)) + .eval(FILE); + assertThat(shouldRead).as("greaterThanOrEqual nested column should not match").isFalse(); + + shouldRead = + new StrictMetricsEvaluator( + SCHEMA, lessThanOrEqual("struct.nested_col_with_stats", INT_MAX_VALUE)) + .eval(FILE); + assertThat(shouldRead).as("lessThanOrEqual nested column should not match").isFalse(); + + shouldRead = + new StrictMetricsEvaluator(SCHEMA, isNull("struct.nested_col_with_stats")).eval(FILE); + assertThat(shouldRead).as("isNull nested column should not match").isFalse(); + + shouldRead = + new StrictMetricsEvaluator(SCHEMA, notNull("struct.nested_col_with_stats")).eval(FILE); + assertThat(shouldRead).as("notNull nested column should not match").isFalse(); + } } diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestStringLiteralConversions.java b/api/src/test/java/org/apache/iceberg/expressions/TestStringLiteralConversions.java index 1dc2225b3805..45d6654f6aaf 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestStringLiteralConversions.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestStringLiteralConversions.java @@ -132,6 +132,51 @@ public void testStringToTimestampLiteral() { .isEqualTo(avroValue); } + @Test + public void testStringToTimestampLiteralWithMicrosecondPrecisionFromNanoseconds() { + // use Avro's timestamp conversion to validate the result + Schema avroSchema = LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)); + TimeConversions.TimestampMicrosConversion avroConversion = + new TimeConversions.TimestampMicrosConversion(); + + Literal timestampStr = Literal.of("2017-08-18T14:21:01.123456789"); + Literal timestamp = timestampStr.to(Types.TimestampType.withoutZone()); + long avroValue = + avroConversion.toLong( + LocalDateTime.of(2017, 8, 18, 14, 21, 1, 123456000).toInstant(ZoneOffset.UTC), + avroSchema, + avroSchema.getLogicalType()); + + assertThat((long) timestamp.value()) + .as("Timestamp without zone should match UTC") + .isEqualTo(avroValue); + } + + @Test + public void testStringToTimestampLiteralWithNanosecondPrecisionFromNanoseconds() { + Literal timestampStr = Literal.of("2017-08-18T14:21:01.123456789"); + Literal timestamp = timestampStr.to(Types.TimestampNanoType.withoutZone()); + + // Not only using Avro's timestamp conversion as it has no timestampNanos(). + long expected = 1503066061123456789L; + assertThat((long) timestamp.value()) + .as("Timestamp without zone should match UTC") + .isEqualTo(expected); + + // use Avro's timestamp conversion to validate the result within one microsecond + Schema avroSchema = LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)); + TimeConversions.TimestampMicrosConversion avroConversion = + new TimeConversions.TimestampMicrosConversion(); + long avroValue = + avroConversion.toLong( + LocalDateTime.of(2017, 8, 18, 14, 21, 1, 123456000).toInstant(ZoneOffset.UTC), + avroSchema, + avroSchema.getLogicalType()); + assertThat(timestamp.value() - avroValue * 1000) + .as("Timestamp without zone should match UTC") + .isEqualTo(789L); + } + @Test public void testNegativeStringToTimestampLiteral() { // use Avro's timestamp conversion to validate the result @@ -181,8 +226,13 @@ public void testNegativeStringToTimestampLiteral() { @Test public void testTimestampWithZoneWithoutZoneInLiteral() { // Zone must be present in literals when converting to timestamp with zone - Literal timestampStr = Literal.of("2017-08-18T14:21:01.919"); - assertThatThrownBy(() -> timestampStr.to(Types.TimestampType.withZone())) + assertThatThrownBy( + () -> Literal.of("2017-08-18T14:21:01.919").to(Types.TimestampType.withZone())) + .isInstanceOf(DateTimeException.class) + .hasMessageContaining("could not be parsed"); + assertThatThrownBy( + () -> + Literal.of("2017-08-18T14:21:01.919123456").to(Types.TimestampNanoType.withZone())) .isInstanceOf(DateTimeException.class) .hasMessageContaining("could not be parsed"); } @@ -190,8 +240,14 @@ public void testTimestampWithZoneWithoutZoneInLiteral() { @Test public void testTimestampWithoutZoneWithZoneInLiteral() { // Zone must not be present in literals when converting to timestamp without zone - Literal timestampStr = Literal.of("2017-08-18T14:21:01.919+07:00"); - assertThatThrownBy(() -> timestampStr.to(Types.TimestampType.withoutZone())) + assertThatThrownBy( + () -> Literal.of("2017-08-18T14:21:01.919+07:00").to(Types.TimestampType.withoutZone())) + .isInstanceOf(DateTimeException.class) + .hasMessageContaining("could not be parsed"); + assertThatThrownBy( + () -> + Literal.of("2017-08-18T14:21:01.919123456+07:00") + .to(Types.TimestampNanoType.withoutZone())) .isInstanceOf(DateTimeException.class) .hasMessageContaining("could not be parsed"); } diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestTimestampLiteralConversions.java b/api/src/test/java/org/apache/iceberg/expressions/TestTimestampLiteralConversions.java new file mode 100644 index 000000000000..379ad4db5e97 --- /dev/null +++ b/api/src/test/java/org/apache/iceberg/expressions/TestTimestampLiteralConversions.java @@ -0,0 +1,245 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.expressions; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.time.format.DateTimeParseException; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.DateTimeUtil; +import org.junit.jupiter.api.Test; + +public class TestTimestampLiteralConversions { + @Test + public void testTimestampToTimestampNanoConversion() { + Literal timestamp = + Literal.of("2017-11-16T14:31:08.000000001").to(Types.TimestampType.withoutZone()); + assertThat(timestamp.value()).isEqualTo(1510842668000000L); + + Literal timestampNano = timestamp.to(Types.TimestampNanoType.withoutZone()); + assertThat(timestampNano.value()).isEqualTo(1510842668000000000L); + + timestamp = Literal.of("1970-01-01T00:00:00.000000001").to(Types.TimestampType.withoutZone()); + assertThat(timestamp.value()).isEqualTo(0L); + + timestampNano = timestamp.to(Types.TimestampNanoType.withoutZone()); + assertThat(timestampNano.value()).isEqualTo(0L); + + timestamp = Literal.of("1969-12-31T23:59:59.999999999").to(Types.TimestampType.withoutZone()); + assertThat(timestamp.value()).isEqualTo(0L); + + timestampNano = timestamp.to(Types.TimestampNanoType.withoutZone()); + assertThat(timestampNano.value()).isEqualTo(0L); + + timestamp = Literal.of("1969-12-31T23:59:59.999999000").to(Types.TimestampType.withoutZone()); + assertThat(timestamp.value()).isEqualTo(-1L); + + timestampNano = timestamp.to(Types.TimestampNanoType.withoutZone()); + assertThat(timestampNano.value()).isEqualTo(-1000L); + } + + @Test + public void testTimestampToDateConversion() { + Literal ts = + Literal.of("2017-11-16T14:31:08.000001").to(Types.TimestampType.withoutZone()); + int dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(DateTimeUtil.isoDateToDays("2017-11-16")); + + ts = Literal.of("1970-01-01T00:00:00.000001").to(Types.TimestampType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(0); + + ts = Literal.of("1969-12-31T23:59:59.999999").to(Types.TimestampType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(-1); + + ts = Literal.of("2017-11-16T14:31:08.000000001").to(Types.TimestampType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(DateTimeUtil.isoDateToDays("2017-11-16")); + + ts = Literal.of("1970-01-01T00:00:00.000000001").to(Types.TimestampType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(0).isEqualTo(DateTimeUtil.isoDateToDays("1970-01-01")); + + ts = Literal.of("1969-12-31T23:59:59.999999999").to(Types.TimestampType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(0).isEqualTo(DateTimeUtil.isoDateToDays("1970-01-01")); + + ts = Literal.of("1969-12-31T23:59:59.999999000").to(Types.TimestampType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(-1).isEqualTo(DateTimeUtil.isoDateToDays("1969-12-31")); + } + + @Test + public void testTimestampMicrosToDateConversion() { + Literal ts = + Literal.of("2017-11-16T14:31:08.000000001").to(Types.TimestampType.withoutZone()); + int dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(DateTimeUtil.isoDateToDays("2017-11-16")); + + ts = Literal.of("1970-01-01T00:00:00.000000001").to(Types.TimestampType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(0); + + ts = Literal.of("1969-12-31T23:59:59.999999999").to(Types.TimestampType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(0); + + ts = Literal.of("1969-12-31T23:59:59.999999000").to(Types.TimestampType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(-1); + } + + @Test + public void testTimestampNanoToTimestampConversion() { + Literal timestamp = + Literal.of("2017-11-16T14:31:08.000000001").to(Types.TimestampNanoType.withoutZone()); + assertThat(timestamp.value()).isEqualTo(1510842668000000001L); + + Literal timestampNano = timestamp.to(Types.TimestampType.withoutZone()); + assertThat(timestampNano.value()).isEqualTo(1510842668000000L); + + timestamp = + Literal.of("1970-01-01T00:00:00.000000001").to(Types.TimestampNanoType.withoutZone()); + assertThat(timestamp.value()).isEqualTo(1L); + + timestampNano = timestamp.to(Types.TimestampType.withoutZone()); + assertThat(timestampNano.value()).isEqualTo(0L); + + timestamp = + Literal.of("1969-12-31T23:59:59.999999999").to(Types.TimestampNanoType.withoutZone()); + assertThat(timestamp.value()).isEqualTo(-1L); + + timestampNano = timestamp.to(Types.TimestampType.withoutZone()); + assertThat(timestampNano.value()).isEqualTo(-1L); + + timestamp = + Literal.of("1969-12-31T23:59:59.999999000").to(Types.TimestampNanoType.withoutZone()); + assertThat(timestamp.value()).isEqualTo(-1000L); + + timestampNano = timestamp.to(Types.TimestampType.withoutZone()); + assertThat(timestampNano.value()).isEqualTo(-1L); + } + + @Test + public void testTimestampNanosToDateConversion() { + Literal ts = + Literal.of("2017-11-16T14:31:08.000000001").to(Types.TimestampNanoType.withoutZone()); + int dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(DateTimeUtil.isoDateToDays("2017-11-16")); + + ts = Literal.of("1970-01-01T00:00:00.000000001").to(Types.TimestampNanoType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(0); + + ts = Literal.of("1969-12-31T23:59:59.999999999").to(Types.TimestampNanoType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(-1); + + ts = Literal.of("1969-12-31T23:59:59.999999000").to(Types.TimestampNanoType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(-1); + } + + @Test + public void testTimestampNanosWithZoneConversion() { + Literal isoTimestampNanosWithZoneOffset = + Literal.of("2017-11-16T14:31:08.000000001+00:00"); + + assertThatThrownBy(() -> isoTimestampNanosWithZoneOffset.to(Types.TimestampType.withoutZone())) + .as("Should not convert timestamp with offset to a timestamp without zone") + .isInstanceOf(DateTimeParseException.class); + + assertThatThrownBy( + () -> isoTimestampNanosWithZoneOffset.to(Types.TimestampNanoType.withoutZone())) + .as("Should not convert timestamp with offset to a timestamp without zone") + .isInstanceOf(DateTimeParseException.class); + + assertThat(isoTimestampNanosWithZoneOffset.to(Types.TimestampType.withZone()).value()) + .isEqualTo(1510842668000000L); + + assertThat(isoTimestampNanosWithZoneOffset.to(Types.TimestampNanoType.withZone()).value()) + .isEqualTo(1510842668000000001L); + } + + @Test + public void testTimestampMicrosWithZoneConversion() { + Literal isoTimestampMicrosWithZoneOffset = + Literal.of("2017-11-16T14:31:08.000001+00:00"); + + assertThatThrownBy(() -> isoTimestampMicrosWithZoneOffset.to(Types.TimestampType.withoutZone())) + .as("Should not convert timestamp with offset to a timestamp without zone") + .isInstanceOf(DateTimeParseException.class); + + assertThatThrownBy( + () -> isoTimestampMicrosWithZoneOffset.to(Types.TimestampNanoType.withoutZone())) + .as("Should not convert timestamp with offset to a timestamp without zone") + .isInstanceOf(DateTimeParseException.class); + + assertThat(isoTimestampMicrosWithZoneOffset.to(Types.TimestampType.withZone()).value()) + .isEqualTo(1510842668000001L); + + assertThat(isoTimestampMicrosWithZoneOffset.to(Types.TimestampNanoType.withZone()).value()) + .isEqualTo(1510842668000001000L); + } + + @Test + public void testTimestampNanosWithoutZoneConversion() { + Literal isoTimestampNanosWithoutZoneOffset = + Literal.of("2017-11-16T14:31:08.000000001"); + + assertThatThrownBy(() -> isoTimestampNanosWithoutZoneOffset.to(Types.TimestampType.withZone())) + .as("Should not convert timestamp without offset to a timestamp with zone") + .isInstanceOf(DateTimeParseException.class); + + assertThatThrownBy( + () -> isoTimestampNanosWithoutZoneOffset.to(Types.TimestampNanoType.withZone())) + .as("Should not convert timestamp without offset to a timestamp with zone") + .isInstanceOf(DateTimeParseException.class); + + assertThat(isoTimestampNanosWithoutZoneOffset.to(Types.TimestampType.withoutZone()).value()) + .isEqualTo(1510842668000000L); + + assertThat(isoTimestampNanosWithoutZoneOffset.to(Types.TimestampNanoType.withoutZone()).value()) + .isEqualTo(1510842668000000001L); + } + + @Test + public void testTimestampMicrosWithoutZoneConversion() { + Literal isoTimestampMicrosWithoutZoneOffset = + Literal.of("2017-11-16T14:31:08.000001"); + + assertThatThrownBy(() -> isoTimestampMicrosWithoutZoneOffset.to(Types.TimestampType.withZone())) + .as("Should not convert timestamp without offset to a timestamp with zone") + .isInstanceOf(DateTimeParseException.class); + + assertThatThrownBy( + () -> isoTimestampMicrosWithoutZoneOffset.to(Types.TimestampNanoType.withZone())) + .as("Should not convert timestamp without offset to a timestamp with zone") + .isInstanceOf(DateTimeParseException.class); + + assertThat(isoTimestampMicrosWithoutZoneOffset.to(Types.TimestampType.withoutZone()).value()) + .isEqualTo(1510842668000001L); + + assertThat( + isoTimestampMicrosWithoutZoneOffset.to(Types.TimestampNanoType.withoutZone()).value()) + .isEqualTo(1510842668000001000L); + } +} diff --git a/api/src/test/java/org/apache/iceberg/transforms/TestBucketing.java b/api/src/test/java/org/apache/iceberg/transforms/TestBucketing.java index 28d01efa6d3b..fc4333d7c6c5 100644 --- a/api/src/test/java/org/apache/iceberg/transforms/TestBucketing.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestBucketing.java @@ -165,6 +165,62 @@ public void testLong() { .isEqualTo(hashBytes(buffer.array())); } + @Test + public void testTimestampNanoPromotion() { + Types.TimestampType tsType = Types.TimestampType.withoutZone(); + Types.TimestampNanoType tsNsType = Types.TimestampNanoType.withoutZone(); + Bucket tsNsBucket = Bucket.get(tsNsType, 1); + Bucket tsBucket = Bucket.get(tsType, 1); + + // Values from spec Appendix B: 32-bit Hash Requirements + assertThat(tsBucket.hash(Literal.of("2017-11-16T22:31:08").to(tsType).value())) + .as( + "Spec example: hash(2017-11-16T22:31:08) = -2047944441 for Timestamp and TimestampNano should match") + .isEqualTo(-2047944441); + assertThat(tsNsBucket.hash(Literal.of("2017-11-16T22:31:08").to(tsNsType).value())) + .as( + "Spec example: hash(2017-11-16T22:31:08) = -2047944441 for Timestamp and TimestampNano should match") + .isEqualTo(-2047944441); + + assertThat(tsBucket.hash(Literal.of("2017-11-16T22:31:08.000001").to(tsType).value())) + .as( + "Spec example: hash(2017-11-16T22:31:08.000001) = -1207196810 for Timestamp and TimestampNano should match") + .isEqualTo(-1207196810); + assertThat(tsNsBucket.hash(Literal.of("2017-11-16T22:31:08.000001001").to(tsNsType).value())) + .as( + "Spec example: hash(2017-11-16T22:31:08.000001) = -1207196810 for Timestamp and TimestampNano should match") + .isEqualTo(-1207196810); + } + + @Test + public void testTimestampTzNanoPromotion() { + Types.TimestampType tsTzType = Types.TimestampType.withZone(); + Types.TimestampNanoType tsTzNsType = Types.TimestampNanoType.withZone(); + Bucket tsTzNsBucket = Bucket.get(tsTzNsType, 1); + Bucket tsTzBucket = Bucket.get(tsTzType, 1); + + // Values from spec Appendix B: 32-bit Hash Requirements + assertThat(tsTzBucket.hash(Literal.of("2017-11-16T14:31:08-08:00").to(tsTzType).value())) + .as( + "Spec example: hash(2017-11-16T14:31:08-08:00) = -2047944441 for Timestamp and TimestampNano should match") + .isEqualTo(-2047944441); + assertThat(tsTzNsBucket.hash(Literal.of("2017-11-16T14:31:08-08:00").to(tsTzNsType).value())) + .as( + "Spec example: hash(2017-11-16T14:31:08-08:00) = -2047944441 for Timestamp and TimestampNano should match") + .isEqualTo(-2047944441); + + assertThat(tsTzBucket.hash(Literal.of("2017-11-16T14:31:08.000001-08:00").to(tsTzType).value())) + .as( + "Spec example: hash(2017-11-16T14:31:08.000001-08:00) = -1207196810 for Timestamp and TimestampNano should match") + .isEqualTo(-1207196810); + assertThat( + tsTzNsBucket.hash( + Literal.of("2017-11-16T14:31:08.000001001-08:00").to(tsTzNsType).value())) + .as( + "Spec example: hash(2017-11-16T14:31:08.000001-08:00) = -1207196810 for Timestamp and TimestampNano should match") + .isEqualTo(-1207196810); + } + @Test public void testIntegerTypePromotion() { int randomInt = testRandom.nextInt(); diff --git a/api/src/test/java/org/apache/iceberg/transforms/TestDates.java b/api/src/test/java/org/apache/iceberg/transforms/TestDates.java index b9c380244666..c899b4cfa1cb 100644 --- a/api/src/test/java/org/apache/iceberg/transforms/TestDates.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestDates.java @@ -26,6 +26,75 @@ import org.junit.jupiter.api.Test; public class TestDates { + @Test + public void testSatisfiesOrderOfDates() { + assertThat(Dates.DAY.satisfiesOrderOf(Dates.DAY)).isTrue(); + assertThat(Dates.DAY.satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Dates.DAY.satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Dates.MONTH.satisfiesOrderOf(Dates.DAY)).isFalse(); + assertThat(Dates.MONTH.satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Dates.MONTH.satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Dates.YEAR.satisfiesOrderOf(Dates.DAY)).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Dates.MONTH)).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Dates.YEAR)).isTrue(); + } + + @Test + public void testSatisfiesOrderOfTimestamps() { + assertThat(Dates.DAY.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Dates.DAY.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isTrue(); + assertThat(Dates.DAY.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Dates.DAY.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Dates.MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Dates.MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isFalse(); + assertThat(Dates.MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Dates.MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Dates.YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + } + + @Test + public void testSatisfiesOrderOfTimestampNanos() { + assertThat(Dates.DAY.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Dates.DAY.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isTrue(); + assertThat(Dates.DAY.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Dates.DAY.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Dates.MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Dates.MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isFalse(); + assertThat(Dates.MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Dates.MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Dates.YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + } + + @Test + public void testSatisfiesOrderOfTimeTransforms() { + assertThat(Dates.DAY.satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Dates.DAY.satisfiesOrderOf(Days.get())).isTrue(); + assertThat(Dates.DAY.satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Dates.DAY.satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Dates.MONTH.satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Dates.MONTH.satisfiesOrderOf(Days.get())).isFalse(); + assertThat(Dates.MONTH.satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Dates.MONTH.satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Dates.YEAR.satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Days.get())).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Months.get())).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Years.get())).isTrue(); + } + @Test @SuppressWarnings("deprecation") public void testDeprecatedDateTransform() { diff --git a/api/src/test/java/org/apache/iceberg/transforms/TestIdentity.java b/api/src/test/java/org/apache/iceberg/transforms/TestIdentity.java index 6101fdf0986d..93d3281411f3 100644 --- a/api/src/test/java/org/apache/iceberg/transforms/TestIdentity.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestIdentity.java @@ -106,7 +106,7 @@ public void testTimestampWithZoneHumanString() { // value will always be in UTC assertThat(identity.toHumanString(timestamptz, ts.value())) .as("Should produce timestamp with time zone adjusted to UTC") - .isEqualTo("2017-12-01T18:12:55.038194Z"); + .isEqualTo("2017-12-01T18:12:55.038194+00:00"); } @Test diff --git a/api/src/test/java/org/apache/iceberg/transforms/TestTimeTransforms.java b/api/src/test/java/org/apache/iceberg/transforms/TestTimeTransforms.java new file mode 100644 index 000000000000..2a161f9bc822 --- /dev/null +++ b/api/src/test/java/org/apache/iceberg/transforms/TestTimeTransforms.java @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.transforms; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestTimeTransforms { + @Test + public void testMicrosSatisfiesOrderOfDates() { + assertThat(Hours.get().satisfiesOrderOf(Dates.DAY)).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Days.get().satisfiesOrderOf(Dates.DAY)).isTrue(); + assertThat(Days.get().satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Days.get().satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Months.get().satisfiesOrderOf(Dates.DAY)).isFalse(); + assertThat(Months.get().satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Months.get().satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Years.get().satisfiesOrderOf(Dates.DAY)).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Dates.MONTH)).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Dates.YEAR)).isTrue(); + } + + @Test + public void testMicrosSatisfiesOrderOfTimestamps() { + assertThat(Hours.get().satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Days.get().satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Days.get().satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isTrue(); + assertThat(Days.get().satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Days.get().satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Months.get().satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Months.get().satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isFalse(); + assertThat(Months.get().satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Months.get().satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Years.get().satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + } + + @Test + public void testMicrosSatisfiesOrderOfTimestampNanos() { + assertThat(Hours.get().satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Days.get().satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Days.get().satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isTrue(); + assertThat(Days.get().satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Days.get().satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Months.get().satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Months.get().satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isFalse(); + assertThat(Months.get().satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Months.get().satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Years.get().satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + } + + @Test + public void testMicrosSatisfiesOrderOfTimeTransforms() { + assertThat(Hours.get().satisfiesOrderOf(Hours.get())).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Days.get())).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Days.get().satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Days.get().satisfiesOrderOf(Days.get())).isTrue(); + assertThat(Days.get().satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Days.get().satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Months.get().satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Months.get().satisfiesOrderOf(Days.get())).isFalse(); + assertThat(Months.get().satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Months.get().satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Years.get().satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Days.get())).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Months.get())).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Years.get())).isTrue(); + } + + @Test + public void testHoursToEnum() { + Hours hours = Hours.get(); + Types.DateType type = Types.DateType.get(); + assertThatThrownBy(() -> hours.toEnum(type)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageMatching("Unsupported type: date"); + } +} diff --git a/api/src/test/java/org/apache/iceberg/transforms/TestTimestamps.java b/api/src/test/java/org/apache/iceberg/transforms/TestTimestamps.java index 3c37e643eb95..78b0e67c686b 100644 --- a/api/src/test/java/org/apache/iceberg/transforms/TestTimestamps.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestTimestamps.java @@ -26,13 +26,222 @@ import org.junit.jupiter.api.Test; public class TestTimestamps { + @Test + public void testMicrosSatisfiesOrderOfDates() { + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Dates.DAY)).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Dates.DAY)).isTrue(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Dates.DAY)).isFalse(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Dates.DAY)).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Dates.MONTH)).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Dates.YEAR)).isTrue(); + } + + @Test + public void testMicrosSatisfiesOrderOfTimestamps() { + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isTrue(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isFalse(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + } + + @Test + public void testMicrosSatisfiesOrderOfTimestampNanos() { + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isTrue(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isFalse(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + } + + @Test + public void testMicrosSatisfiesOrderOfTimeTransforms() { + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Hours.get())).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Days.get())).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Days.get())).isTrue(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Days.get())).isFalse(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Days.get())).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Months.get())).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Years.get())).isTrue(); + } + + @Test + public void testNanosSatisfiesOrderOfDates() { + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Dates.DAY)).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Dates.DAY)).isTrue(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Dates.DAY)).isFalse(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Dates.DAY)).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Dates.MONTH)).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Dates.YEAR)).isTrue(); + } + + @Test + public void testNanosSatisfiesOrderOfTimestamps() { + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isTrue(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isFalse(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + } + + @Test + public void testNanosSatisfiesOrderOfTimestampNanos() { + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isTrue(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isFalse(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + } + + @Test + public void testNanosSatisfiesOrderOfTimeTransforms() { + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Hours.get())).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Days.get())).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Days.get())).isTrue(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Days.get())).isFalse(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Days.get())).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Months.get())).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Years.get())).isTrue(); + } + @Test @SuppressWarnings("deprecation") public void testDeprecatedTimestampTransform() { Types.TimestampType type = Types.TimestampType.withoutZone(); Literal ts = Literal.of("2017-12-01T10:12:55.038194").to(type); Literal pts = Literal.of("1970-01-01T00:00:01.000001").to(type); - Literal nts = Literal.of("1969-12-31T23:59:58.999999").to(type); + Literal nts = Literal.of("1969-12-31T23:59:59.999999").to(type); + + Transform years = Transforms.year(type); + assertThat((int) years.apply(ts.value())).as("Should produce 2017 - 1970 = 47").isEqualTo(47); + assertThat((int) years.apply(pts.value())).as("Should produce 1970 - 1970 = 0").isZero(); + assertThat((int) years.apply(nts.value())).as("Should produce 1969 - 1970 = -1").isEqualTo(-1); + + Transform months = Transforms.month(type); + assertThat((int) months.apply(ts.value())) + .as("Should produce 47 * 12 + 11 = 575") + .isEqualTo(575); + assertThat((int) months.apply(pts.value())).as("Should produce 0 * 12 + 0 = 0").isZero(); + assertThat((int) months.apply(nts.value())).isEqualTo(-1); + + Transform days = Transforms.day(type); + assertThat((int) days.apply(ts.value())).as("Should produce 17501").isEqualTo(17501); + assertThat((int) days.apply(pts.value())).as("Should produce 0 * 365 + 0 = 0").isZero(); + assertThat((int) days.apply(nts.value())).isEqualTo(-1); + + Transform hours = Transforms.hour(type); + assertThat((int) hours.apply(ts.value())) + .as("Should produce 17501 * 24 + 10") + .isEqualTo(420034); + assertThat((int) hours.apply(pts.value())).as("Should produce 0 * 24 + 0 = 0").isZero(); + assertThat((int) hours.apply(nts.value())).isEqualTo(-1); + } + + @Test + @SuppressWarnings("deprecation") + public void testDeprecatedTimestampNanoTransform() { + Types.TimestampNanoType type = Types.TimestampNanoType.withoutZone(); + Literal ts = Literal.of("2017-12-01T10:12:55.038194789").to(type); + Literal pts = Literal.of("1970-01-01T00:00:01.000000001").to(type); + Literal nts = Literal.of("1969-12-31T23:59:59.999999999").to(type); Transform years = Transforms.year(type); assertThat((int) years.apply(ts.value())).as("Should produce 2017 - 1970 = 47").isEqualTo(47); @@ -64,7 +273,51 @@ public void testTimestampTransform() { Types.TimestampType type = Types.TimestampType.withoutZone(); Literal ts = Literal.of("2017-12-01T10:12:55.038194").to(type); Literal pts = Literal.of("1970-01-01T00:00:01.000001").to(type); - Literal nts = Literal.of("1969-12-31T23:59:58.999999").to(type); + Literal nts = Literal.of("1969-12-31T23:59:59.999999").to(type); + + Transform years = Transforms.year(); + assertThat((int) years.bind(type).apply(ts.value())) + .as("Should produce 2017 - 1970 = 47") + .isEqualTo(47); + assertThat((int) years.bind(type).apply(pts.value())) + .as("Should produce 1970 - 1970 = 0") + .isZero(); + assertThat((int) years.bind(type).apply(nts.value())) + .as("Should produce 1969 - 1970 = -1") + .isEqualTo(-1); + + Transform months = Transforms.month(); + assertThat((int) months.bind(type).apply(ts.value())) + .as("Should produce 47 * 12 + 11 = 575") + .isEqualTo(575); + assertThat((int) months.bind(type).apply(pts.value())) + .as("Should produce 0 * 12 + 0 = 0") + .isZero(); + assertThat((int) months.bind(type).apply(nts.value())).isEqualTo(-1); + + Transform days = Transforms.day(); + assertThat((int) days.bind(type).apply(ts.value())).as("Should produce 17501").isEqualTo(17501); + assertThat((int) days.bind(type).apply(pts.value())) + .as("Should produce 0 * 365 + 0 = 0") + .isZero(); + assertThat((int) days.bind(type).apply(nts.value())).isEqualTo(-1); + + Transform hours = Transforms.hour(); + assertThat((int) hours.bind(type).apply(ts.value())) + .as("Should produce 17501 * 24 + 10") + .isEqualTo(420034); + assertThat((int) hours.bind(type).apply(pts.value())) + .as("Should produce 0 * 24 + 0 = 0") + .isZero(); + assertThat((int) hours.bind(type).apply(nts.value())).isEqualTo(-1); + } + + @Test + public void testTimestampNanoTransform() { + Types.TimestampNanoType type = Types.TimestampNanoType.withoutZone(); + Literal ts = Literal.of("2017-12-01T10:12:55.038194789").to(type); + Literal pts = Literal.of("1970-01-01T00:00:01.000000001").to(type); + Literal nts = Literal.of("1969-12-31T23:59:59.999999999").to(type); Transform years = Transforms.year(); assertThat((int) years.bind(type).apply(ts.value())) @@ -123,6 +376,26 @@ public void testTimestampWithoutZoneToHumanString() { .isEqualTo("2017-12-01-10"); } + @Test + public void testTimestampNanoWithoutZoneToHumanString() { + Types.TimestampNanoType type = Types.TimestampNanoType.withoutZone(); + Literal date = Literal.of("2017-12-01T10:12:55.038194789").to(type); + + Transform year = Transforms.year(); + assertThat(year.toHumanString(type, year.bind(type).apply(date.value()))).isEqualTo("2017"); + + Transform month = Transforms.month(); + assertThat(month.toHumanString(type, month.bind(type).apply(date.value()))) + .isEqualTo("2017-12"); + + Transform day = Transforms.day(); + assertThat(day.toHumanString(type, day.bind(type).apply(date.value()))).isEqualTo("2017-12-01"); + + Transform hour = Transforms.hour(); + assertThat(hour.toHumanString(type, hour.bind(type).apply(date.value()))) + .isEqualTo("2017-12-01-10"); + } + @Test public void testNegativeTimestampWithoutZoneToHumanString() { Types.TimestampType type = Types.TimestampType.withoutZone(); @@ -143,6 +416,26 @@ public void testNegativeTimestampWithoutZoneToHumanString() { .isEqualTo("1969-12-30-10"); } + @Test + public void testNegativeTimestampNanoWithoutZoneToHumanString() { + Types.TimestampNanoType type = Types.TimestampNanoType.withoutZone(); + Literal date = Literal.of("1969-12-30T10:12:55.038194789").to(type); + + Transform year = Transforms.year(); + assertThat(year.toHumanString(type, year.bind(type).apply(date.value()))).isEqualTo("1969"); + + Transform month = Transforms.month(); + assertThat(month.toHumanString(type, month.bind(type).apply(date.value()))) + .isEqualTo("1969-12"); + + Transform day = Transforms.day(); + assertThat(day.toHumanString(type, day.bind(type).apply(date.value()))).isEqualTo("1969-12-30"); + + Transform hour = Transforms.hour(); + assertThat(hour.toHumanString(type, hour.bind(type).apply(date.value()))) + .isEqualTo("1969-12-30-10"); + } + @Test public void testNegativeTimestampWithoutZoneToHumanStringLowerBound() { Types.TimestampType type = Types.TimestampType.withoutZone(); @@ -163,6 +456,26 @@ public void testNegativeTimestampWithoutZoneToHumanStringLowerBound() { .isEqualTo("1969-12-30-00"); } + @Test + public void testNegativeTimestampNanoWithoutZoneToHumanStringLowerBound() { + Types.TimestampNanoType type = Types.TimestampNanoType.withoutZone(); + Literal date = Literal.of("1969-12-30T00:00:00.000000000").to(type); + + Transform year = Transforms.year(); + assertThat(year.toHumanString(type, year.bind(type).apply(date.value()))).isEqualTo("1969"); + + Transform month = Transforms.month(); + assertThat(month.toHumanString(type, month.bind(type).apply(date.value()))) + .isEqualTo("1969-12"); + + Transform day = Transforms.day(); + assertThat(day.toHumanString(type, day.bind(type).apply(date.value()))).isEqualTo("1969-12-30"); + + Transform hour = Transforms.hour(); + assertThat(hour.toHumanString(type, hour.bind(type).apply(date.value()))) + .isEqualTo("1969-12-30-00"); + } + @Test public void testNegativeTimestampWithoutZoneToHumanStringUpperBound() { Types.TimestampType type = Types.TimestampType.withoutZone(); @@ -183,6 +496,26 @@ public void testNegativeTimestampWithoutZoneToHumanStringUpperBound() { .isEqualTo("1969-12-31-23"); } + @Test + public void testNegativeTimestampNanoWithoutZoneToHumanStringUpperBound() { + Types.TimestampNanoType type = Types.TimestampNanoType.withoutZone(); + Literal date = Literal.of("1969-12-31T23:59:59.999999999").to(type); + + Transform year = Transforms.year(); + assertThat(year.toHumanString(type, year.bind(type).apply(date.value()))).isEqualTo("1969"); + + Transform month = Transforms.month(); + assertThat(month.toHumanString(type, month.bind(type).apply(date.value()))) + .isEqualTo("1969-12"); + + Transform day = Transforms.day(); + assertThat(day.toHumanString(type, day.bind(type).apply(date.value()))).isEqualTo("1969-12-31"); + + Transform hour = Transforms.hour(); + assertThat(hour.toHumanString(type, hour.bind(type).apply(date.value()))) + .isEqualTo("1969-12-31-23"); + } + @Test public void testTimestampWithZoneToHumanString() { Types.TimestampType type = Types.TimestampType.withZone(); @@ -205,7 +538,28 @@ public void testTimestampWithZoneToHumanString() { } @Test - public void testNullHumanString() { + public void testTimestampNanoWithZoneToHumanString() { + Types.TimestampNanoType type = Types.TimestampNanoType.withZone(); + Literal date = Literal.of("2017-12-01T10:12:55.038194789-08:00").to(type); + + Transform year = Transforms.year(); + assertThat(year.toHumanString(type, year.bind(type).apply(date.value()))).isEqualTo("2017"); + + Transform month = Transforms.month(); + assertThat(month.toHumanString(type, month.bind(type).apply(date.value()))) + .isEqualTo("2017-12"); + + Transform day = Transforms.day(); + assertThat(day.toHumanString(type, day.bind(type).apply(date.value()))).isEqualTo("2017-12-01"); + + // the hour is 18 because the value is always UTC + Transform hour = Transforms.hour(); + assertThat(hour.toHumanString(type, hour.bind(type).apply(date.value()))) + .isEqualTo("2017-12-01-18"); + } + + @Test + public void testTimestampNullHumanString() { Types.TimestampType type = Types.TimestampType.withZone(); assertThat(Transforms.year().toHumanString(type, null)) .as("Should produce \"null\" for null") @@ -221,6 +575,23 @@ public void testNullHumanString() { .isEqualTo("null"); } + @Test + public void testTimestampNanoNullHumanString() { + Types.TimestampNanoType type = Types.TimestampNanoType.withZone(); + assertThat(Transforms.year().toHumanString(type, null)) + .as("Should produce \"null\" for null") + .isEqualTo("null"); + assertThat(Transforms.month().toHumanString(type, null)) + .as("Should produce \"null\" for null") + .isEqualTo("null"); + assertThat(Transforms.day().toHumanString(type, null)) + .as("Should produce \"null\" for null") + .isEqualTo("null"); + assertThat(Transforms.hour().toHumanString(type, null)) + .as("Should produce \"null\" for null") + .isEqualTo("null"); + } + @Test public void testTimestampsReturnType() { Types.TimestampType type = Types.TimestampType.withZone(); @@ -241,4 +612,25 @@ public void testTimestampsReturnType() { Type hourResultType = hour.getResultType(type); assertThat(hourResultType).isEqualTo(Types.IntegerType.get()); } + + @Test + public void testTimestampNanosReturnType() { + Types.TimestampNanoType type = Types.TimestampNanoType.withZone(); + + Transform year = Transforms.year(); + Type yearResultType = year.getResultType(type); + assertThat(yearResultType).isEqualTo(Types.IntegerType.get()); + + Transform month = Transforms.month(); + Type monthResultType = month.getResultType(type); + assertThat(monthResultType).isEqualTo(Types.IntegerType.get()); + + Transform day = Transforms.day(); + Type dayResultType = day.getResultType(type); + assertThat(dayResultType).isEqualTo(Types.DateType.get()); + + Transform hour = Transforms.hour(); + Type hourResultType = hour.getResultType(type); + assertThat(hourResultType).isEqualTo(Types.IntegerType.get()); + } } diff --git a/api/src/test/java/org/apache/iceberg/types/TestComparators.java b/api/src/test/java/org/apache/iceberg/types/TestComparators.java index 165d96c029cc..07653ba3c8a8 100644 --- a/api/src/test/java/org/apache/iceberg/types/TestComparators.java +++ b/api/src/test/java/org/apache/iceberg/types/TestComparators.java @@ -79,6 +79,12 @@ public void testTimestamp() { assertComparesCorrectly(Comparators.forType(Types.TimestampType.withZone()), 111, 222); } + @Test + public void testTimestampNanos() { + assertComparesCorrectly(Comparators.forType(Types.TimestampNanoType.withoutZone()), 111, 222); + assertComparesCorrectly(Comparators.forType(Types.TimestampNanoType.withZone()), 111, 222); + } + @Test public void testString() { assertComparesCorrectly(Comparators.forType(Types.StringType.get()), "a", "b"); diff --git a/api/src/test/java/org/apache/iceberg/types/TestConversions.java b/api/src/test/java/org/apache/iceberg/types/TestConversions.java index 6c7a884a5839..e207cfd8d59a 100644 --- a/api/src/test/java/org/apache/iceberg/types/TestConversions.java +++ b/api/src/test/java/org/apache/iceberg/types/TestConversions.java @@ -37,6 +37,7 @@ import org.apache.iceberg.types.Types.LongType; import org.apache.iceberg.types.Types.StringType; import org.apache.iceberg.types.Types.TimeType; +import org.apache.iceberg.types.Types.TimestampNanoType; import org.apache.iceberg.types.Types.TimestampType; import org.apache.iceberg.types.Types.UUIDType; import org.junit.jupiter.api.Test; @@ -93,7 +94,7 @@ public void testByteBufferConversions() { assertThat(Literal.of(10000L).to(TimeType.get()).toByteBuffer().array()) .isEqualTo(new byte[] {16, 39, 0, 0, 0, 0, 0, 0}); - // timestamps are stored as microseconds from 1970-01-01 00:00:00.000000 in an 8-byte + // timestamps are stored as micro|nanoseconds from 1970-01-01 00:00:00 in an 8-byte // little-endian long // 400000L is 0...110|00011010|10000000 in binary // 10000000 -> -128, 00011010 -> 26, 00000110 -> 6, ... , 00000000 -> 0 @@ -103,6 +104,16 @@ public void testByteBufferConversions() { .isEqualTo(new byte[] {-128, 26, 6, 0, 0, 0, 0, 0}); assertThat(Literal.of(400000L).to(TimestampType.withZone()).toByteBuffer().array()) .isEqualTo(new byte[] {-128, 26, 6, 0, 0, 0, 0, 0}); + // values passed to assertConversion and Literal.of differ because Literal.of(...) assumes + // the value is in micros, which gets converted when to(TimestampNanoType) is called + assertConversion( + 400000000L, TimestampNanoType.withoutZone(), new byte[] {0, -124, -41, 23, 0, 0, 0, 0}); + assertConversion( + 400000000L, TimestampNanoType.withZone(), new byte[] {0, -124, -41, 23, 0, 0, 0, 0}); + assertThat(Literal.of(400000L).to(TimestampNanoType.withoutZone()).toByteBuffer().array()) + .isEqualTo(new byte[] {0, -124, -41, 23, 0, 0, 0, 0}); + assertThat(Literal.of(400000L).to(TimestampNanoType.withZone()).toByteBuffer().array()) + .isEqualTo(new byte[] {0, -124, -41, 23, 0, 0, 0, 0}); // strings are stored as UTF-8 bytes (without length) // 'A' -> 65, 'B' -> 66, 'C' -> 67 diff --git a/api/src/test/java/org/apache/iceberg/types/TestReadabilityChecks.java b/api/src/test/java/org/apache/iceberg/types/TestReadabilityChecks.java index 7f5948bd5838..2d02da5346a7 100644 --- a/api/src/test/java/org/apache/iceberg/types/TestReadabilityChecks.java +++ b/api/src/test/java/org/apache/iceberg/types/TestReadabilityChecks.java @@ -39,6 +39,8 @@ public class TestReadabilityChecks { Types.TimeType.get(), Types.TimestampType.withoutZone(), Types.TimestampType.withZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.StringType.get(), Types.UUIDType.get(), Types.FixedType.ofLength(3), diff --git a/api/src/test/java/org/apache/iceberg/types/TestSerializableTypes.java b/api/src/test/java/org/apache/iceberg/types/TestSerializableTypes.java index d981b5a26789..97bb2b1a19e4 100644 --- a/api/src/test/java/org/apache/iceberg/types/TestSerializableTypes.java +++ b/api/src/test/java/org/apache/iceberg/types/TestSerializableTypes.java @@ -41,6 +41,8 @@ public void testIdentityTypes() throws Exception { Types.TimeType.get(), Types.TimestampType.withoutZone(), Types.TimestampType.withZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.StringType.get(), Types.UUIDType.get(), Types.BinaryType.get() @@ -110,13 +112,13 @@ public void testMaps() throws Exception { @Test public void testLists() throws Exception { - Type[] maps = + Type[] lists = new Type[] { Types.ListType.ofOptional(2, Types.DoubleType.get()), Types.ListType.ofRequired(5, Types.DoubleType.get()) }; - for (Type list : maps) { + for (Type list : lists) { Type copy = TestHelpers.roundTripSerialize(list); assertThat(copy).as("List serialization should be equal to starting type").isEqualTo(list); assertThat(list.asNestedType().asListType().elementType()) diff --git a/api/src/test/java/org/apache/iceberg/types/TestTypes.java b/api/src/test/java/org/apache/iceberg/types/TestTypes.java index 5db7ca7cd1bc..226c53f1e9ce 100644 --- a/api/src/test/java/org/apache/iceberg/types/TestTypes.java +++ b/api/src/test/java/org/apache/iceberg/types/TestTypes.java @@ -31,6 +31,11 @@ public void fromPrimitiveString() { assertThat(Types.fromPrimitiveString("BooLean")).isSameAs(Types.BooleanType.get()); assertThat(Types.fromPrimitiveString("timestamp")).isSameAs(Types.TimestampType.withoutZone()); + assertThat(Types.fromPrimitiveString("timestamptz")).isSameAs(Types.TimestampType.withZone()); + assertThat(Types.fromPrimitiveString("timestamp_ns")) + .isSameAs(Types.TimestampNanoType.withoutZone()); + assertThat(Types.fromPrimitiveString("timestamptz_ns")) + .isSameAs(Types.TimestampNanoType.withZone()); assertThat(Types.fromPrimitiveString("Fixed[ 3 ]")).isEqualTo(Types.FixedType.ofLength(3)); diff --git a/api/src/test/java/org/apache/iceberg/util/TestDateTimeUtil.java b/api/src/test/java/org/apache/iceberg/util/TestDateTimeUtil.java index 42da3a3dd774..6088fe51b57a 100644 --- a/api/src/test/java/org/apache/iceberg/util/TestDateTimeUtil.java +++ b/api/src/test/java/org/apache/iceberg/util/TestDateTimeUtil.java @@ -20,19 +20,72 @@ import static org.assertj.core.api.Assertions.assertThat; -import java.time.ZonedDateTime; +import org.apache.iceberg.transforms.Transforms; +import org.apache.iceberg.types.Types; import org.junit.jupiter.api.Test; public class TestDateTimeUtil { + @Test + public void nanosToMicros() { + assertThat(DateTimeUtil.nanosToMicros(1510871468000001001L)).isEqualTo(1510871468000001L); + assertThat(DateTimeUtil.nanosToMicros(-1510871468000001001L)).isEqualTo(-1510871468000002L); + } + + @Test + public void microsToNanos() { + assertThat(DateTimeUtil.microsToNanos(1510871468000001L)).isEqualTo(1510871468000001000L); + assertThat(DateTimeUtil.microsToNanos(-1510871468000001L)).isEqualTo(-1510871468000001000L); + } + + @Test + public void isoTimestampToNanos() { + assertThat(DateTimeUtil.isoTimestampToNanos("2017-11-16T22:31:08.000001001")) + .isEqualTo(1510871468000001001L); + assertThat(DateTimeUtil.isoTimestampToNanos("1922-02-15T01:28:51.999998999")) + .isEqualTo(-1510871468000001001L); + } + + @Test + public void isoTimestamptzToNanos() { + assertThat(DateTimeUtil.isoTimestamptzToNanos("2017-11-16T14:31:08.000001001-08:00")) + .isEqualTo(1510871468000001001L); + assertThat(DateTimeUtil.isoTimestamptzToNanos("1922-02-15T01:28:51.999998999+00:00")) + .isEqualTo(-1510871468000001001L); + } + + @Test + public void convertNanos() { + assertThat( + Transforms.identity() + .toHumanString(Types.TimestampNanoType.withoutZone(), 1510871468000001001L)) + .isEqualTo("2017-11-16T22:31:08.000001001"); + assertThat(DateTimeUtil.nanosToYears(1510871468000001001L)).isEqualTo(47); + assertThat(Transforms.year().toHumanString(Types.IntegerType.get(), 47)).isEqualTo("2017"); + assertThat(DateTimeUtil.nanosToMonths(1510871468000001001L)).isEqualTo(574); + assertThat(Transforms.month().toHumanString(Types.IntegerType.get(), 574)).isEqualTo("2017-11"); + assertThat(DateTimeUtil.nanosToDays(1510871468000001001L)).isEqualTo(17486); + assertThat(Transforms.day().toHumanString(Types.IntegerType.get(), 17486)) + .isEqualTo("2017-11-16"); + assertThat(DateTimeUtil.nanosToHours(1510871468000001001L)).isEqualTo(419686); + assertThat(Transforms.hour().toHumanString(Types.IntegerType.get(), 419686)) + .isEqualTo("2017-11-16-22"); + } @Test - public void formatTimestampMillis() { - String timestamp = "1970-01-01T00:00:00.001+00:00"; - assertThat(DateTimeUtil.formatTimestampMillis(1L)).isEqualTo(timestamp); - assertThat(ZonedDateTime.parse(timestamp).toInstant().toEpochMilli()).isEqualTo(1L); - - timestamp = "1970-01-01T00:16:40+00:00"; - assertThat(DateTimeUtil.formatTimestampMillis(1000000L)).isEqualTo(timestamp); - assertThat(ZonedDateTime.parse(timestamp).toInstant().toEpochMilli()).isEqualTo(1000000L); + public void convertNanosNegative() { + assertThat( + Transforms.identity() + .toHumanString(Types.TimestampNanoType.withZone(), -1510871468000001001L)) + .isEqualTo("1922-02-15T01:28:51.999998999+00:00"); + assertThat(DateTimeUtil.nanosToYears(-1510871468000001001L)).isEqualTo(-48); + assertThat(Transforms.year().toHumanString(Types.IntegerType.get(), 47)).isEqualTo("2017"); + assertThat(DateTimeUtil.nanosToMonths(-1510871468000001001L)).isEqualTo(-575); + assertThat(Transforms.month().toHumanString(Types.IntegerType.get(), 574)).isEqualTo("2017-11"); + assertThat(DateTimeUtil.nanosToDays(-1510871468000001001L)).isEqualTo(-17487); + assertThat(Transforms.day().toHumanString(Types.IntegerType.get(), 17486)) + .isEqualTo("2017-11-16"); + assertThat(DateTimeUtil.nanosToHours(-1510871468000001001L)).isEqualTo(-419687); + assertThat(Transforms.hour().toHumanString(Types.IntegerType.get(), 419686)) + .isEqualTo("2017-11-16-22"); } } diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/DecimalVectorUtil.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/DecimalVectorUtil.java index f6fe062b561b..aba051860abf 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/DecimalVectorUtil.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/DecimalVectorUtil.java @@ -19,6 +19,7 @@ package org.apache.iceberg.arrow.vectorized.parquet; import java.util.Arrays; +import java.util.Locale; import org.apache.arrow.vector.DecimalVector; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; @@ -62,7 +63,9 @@ static byte[] padBigEndianBytes(byte[] bigEndianBytes, int newLength) { } throw new IllegalArgumentException( String.format( + Locale.ROOT, "Buffer size of %d is larger than requested size of %d", - bigEndianBytes.length, newLength)); + bigEndianBytes.length, + newLength)); } } diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedColumnIterator.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedColumnIterator.java index 822ca8973f54..54445e424508 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedColumnIterator.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedColumnIterator.java @@ -214,6 +214,10 @@ protected int nextBatchOf( } } + /** + * @deprecated since 1.7.0, will be removed in 1.8.0. + */ + @Deprecated public class FixedWidthTypeBinaryBatchReader extends BatchReader { @Override protected int nextBatchOf( @@ -278,6 +282,10 @@ public VarWidthTypeBatchReader varWidthTypeBatchReader() { return new VarWidthTypeBatchReader(); } + /** + * @deprecated since 1.7.0, will be removed in 1.8.0. + */ + @Deprecated public FixedWidthTypeBinaryBatchReader fixedWidthTypeBinaryBatchReader() { return new FixedWidthTypeBinaryBatchReader(); } diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedDictionaryEncodedParquetValuesReader.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedDictionaryEncodedParquetValuesReader.java index 55f1d3fd7908..848bb2d936c3 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedDictionaryEncodedParquetValuesReader.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedDictionaryEncodedParquetValuesReader.java @@ -22,7 +22,6 @@ import java.nio.ByteOrder; import org.apache.arrow.vector.BaseVariableWidthVector; import org.apache.arrow.vector.BitVectorHelper; -import org.apache.arrow.vector.DecimalVector; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.FixedSizeBinaryVector; import org.apache.arrow.vector.IntVector; @@ -59,14 +58,10 @@ public void nextBatch( } int numValues = Math.min(left, currentCount); for (int i = 0; i < numValues; i++) { - int index = idx * typeWidth; - if (typeWidth == -1) { - index = idx; - } if (Mode.RLE.equals(mode)) { - nextVal(vector, dict, index, currentValue, typeWidth); + nextVal(vector, dict, idx, currentValue, typeWidth); } else if (Mode.PACKED.equals(mode)) { - nextVal(vector, dict, index, packedValuesBuffer[packedValuesBufferIdx++], typeWidth); + nextVal(vector, dict, idx, packedValuesBuffer[packedValuesBufferIdx++], typeWidth); } nullabilityHolder.setNotNull(idx); if (setArrowValidityVector) { @@ -95,7 +90,7 @@ class LongDictEncodedReader extends BaseDictEncodedReader { @Override protected void nextVal( FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) { - vector.getDataBuffer().setLong(idx, dict.decodeToLong(currentVal)); + vector.getDataBuffer().setLong((long) idx * typeWidth, dict.decodeToLong(currentVal)); } } @@ -103,7 +98,7 @@ class TimestampMillisDictEncodedReader extends BaseDictEncodedReader { @Override protected void nextVal( FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) { - vector.getDataBuffer().setLong(idx, dict.decodeToLong(currentVal) * 1000); + vector.getDataBuffer().setLong((long) idx * typeWidth, dict.decodeToLong(currentVal) * 1000); } } @@ -114,7 +109,7 @@ protected void nextVal( ByteBuffer buffer = dict.decodeToBinary(currentVal).toByteBuffer().order(ByteOrder.LITTLE_ENDIAN); long timestampInt96 = ParquetUtil.extractTimestampInt96(buffer); - vector.getDataBuffer().setLong(idx, timestampInt96); + vector.getDataBuffer().setLong((long) idx * typeWidth, timestampInt96); } } @@ -122,7 +117,7 @@ class IntegerDictEncodedReader extends BaseDictEncodedReader { @Override protected void nextVal( FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) { - vector.getDataBuffer().setInt(idx, dict.decodeToInt(currentVal)); + vector.getDataBuffer().setInt((long) idx * typeWidth, dict.decodeToInt(currentVal)); } } @@ -130,7 +125,7 @@ class FloatDictEncodedReader extends BaseDictEncodedReader { @Override protected void nextVal( FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) { - vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(currentVal)); + vector.getDataBuffer().setFloat((long) idx * typeWidth, dict.decodeToFloat(currentVal)); } } @@ -138,25 +133,20 @@ class DoubleDictEncodedReader extends BaseDictEncodedReader { @Override protected void nextVal( FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) { - vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(currentVal)); + vector.getDataBuffer().setDouble((long) idx * typeWidth, dict.decodeToDouble(currentVal)); } } + /** + * @deprecated since 1.7.0, will be removed in 1.8.0. + */ + @Deprecated class FixedWidthBinaryDictEncodedReader extends BaseDictEncodedReader { @Override protected void nextVal( FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) { ByteBuffer buffer = dict.decodeToBinary(currentVal).toByteBuffer(); - vector.getDataBuffer().setBytes(idx, buffer); - } - } - - class FixedLengthDecimalDictEncodedReader extends BaseDictEncodedReader { - @Override - protected void nextVal( - FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) { - byte[] bytes = dict.decodeToBinary(currentVal).getBytesUnsafe(); - DecimalVectorUtil.setBigEndian((DecimalVector) vector, idx, bytes); + vector.getDataBuffer().setBytes((long) idx * typeWidth, buffer); } } @@ -174,22 +164,6 @@ protected void nextVal( } } - class IntBackedDecimalDictEncodedReader extends BaseDictEncodedReader { - @Override - protected void nextVal( - FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) { - ((DecimalVector) vector).set(idx, dict.decodeToInt(currentVal)); - } - } - - class LongBackedDecimalDictEncodedReader extends BaseDictEncodedReader { - @Override - protected void nextVal( - FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) { - ((DecimalVector) vector).set(idx, dict.decodeToLong(currentVal)); - } - } - class FixedSizeBinaryDictEncodedReader extends BaseDictEncodedReader { @Override protected void nextVal( @@ -229,26 +203,18 @@ public DoubleDictEncodedReader doubleDictEncodedReader() { return new DoubleDictEncodedReader(); } + /** + * @deprecated since 1.7.0, will be removed in 1.8.0. + */ + @Deprecated public FixedWidthBinaryDictEncodedReader fixedWidthBinaryDictEncodedReader() { return new FixedWidthBinaryDictEncodedReader(); } - public FixedLengthDecimalDictEncodedReader fixedLengthDecimalDictEncodedReader() { - return new FixedLengthDecimalDictEncodedReader(); - } - public VarWidthBinaryDictEncodedReader varWidthBinaryDictEncodedReader() { return new VarWidthBinaryDictEncodedReader(); } - public IntBackedDecimalDictEncodedReader intBackedDecimalDictEncodedReader() { - return new IntBackedDecimalDictEncodedReader(); - } - - public LongBackedDecimalDictEncodedReader longBackedDecimalDictEncodedReader() { - return new LongBackedDecimalDictEncodedReader(); - } - public FixedSizeBinaryDictEncodedReader fixedSizeBinaryDictEncodedReader() { return new FixedSizeBinaryDictEncodedReader(); } diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedPageIterator.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedPageIterator.java index 7c862d4c3fd6..d36521273d63 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedPageIterator.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedPageIterator.java @@ -421,7 +421,10 @@ protected void nextDictEncodedVal( * Method for reading batches of fixed width binary type (e.g. BYTE[7]). Spark does not support * fixed width binary data type. To work around this limitation, the data is read as fixed width * binary from parquet and stored in a {@link VarBinaryVector} in Arrow. + * + * @deprecated since 1.7.0, will be removed in 1.8.0. */ + @Deprecated class FixedWidthBinaryPageReader extends BasePageReader { @Override protected void nextVal( @@ -496,6 +499,10 @@ VarWidthTypePageReader varWidthTypePageReader() { return new VarWidthTypePageReader(); } + /** + * @deprecated since 1.7.0, will be removed in 1.8.0. + */ + @Deprecated FixedWidthBinaryPageReader fixedWidthBinaryPageReader() { return new FixedWidthBinaryPageReader(); } diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetDefinitionLevelReader.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetDefinitionLevelReader.java index 20d7f804978c..e8ec7bee8f62 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetDefinitionLevelReader.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetDefinitionLevelReader.java @@ -24,7 +24,6 @@ import org.apache.arrow.vector.BaseVariableWidthVector; import org.apache.arrow.vector.BitVector; import org.apache.arrow.vector.BitVectorHelper; -import org.apache.arrow.vector.DecimalVector; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.FixedSizeBinaryVector; import org.apache.arrow.vector.IntVector; @@ -494,6 +493,10 @@ protected void nextDictEncodedVal( } } + /** + * @deprecated since 1.7.0, will be removed in 1.8.0. + */ + @Deprecated class FixedWidthBinaryReader extends BaseReader { @Override protected void nextVal( @@ -532,39 +535,6 @@ protected void nextDictEncodedVal( } } - class FixedLengthDecimalReader extends BaseReader { - @Override - protected void nextVal( - FieldVector vector, - int idx, - ValuesAsBytesReader valuesReader, - int typeWidth, - byte[] byteArray) { - valuesReader.getBuffer(typeWidth).get(byteArray, 0, typeWidth); - DecimalVectorUtil.setBigEndian((DecimalVector) vector, idx, byteArray); - } - - @Override - protected void nextDictEncodedVal( - FieldVector vector, - int idx, - VectorizedDictionaryEncodedParquetValuesReader reader, - int numValuesToRead, - Dictionary dict, - NullabilityHolder nullabilityHolder, - int typeWidth, - Mode mode) { - if (Mode.RLE.equals(mode)) { - reader - .fixedLengthDecimalDictEncodedReader() - .nextBatch(vector, idx, numValuesToRead, dict, nullabilityHolder, typeWidth); - } else if (Mode.PACKED.equals(mode)) { - byte[] bytes = dict.decodeToBinary(reader.readInteger()).getBytesUnsafe(); - DecimalVectorUtil.setBigEndian((DecimalVector) vector, idx, bytes); - } - } - } - class FixedSizeBinaryReader extends BaseReader { @Override protected void nextVal( @@ -645,68 +615,6 @@ protected void nextDictEncodedVal( } } - class IntBackedDecimalReader extends BaseReader { - @Override - protected void nextVal( - FieldVector vector, - int idx, - ValuesAsBytesReader valuesReader, - int typeWidth, - byte[] byteArray) { - ((DecimalVector) vector).set(idx, valuesReader.getBuffer(Integer.BYTES).getInt()); - } - - @Override - protected void nextDictEncodedVal( - FieldVector vector, - int idx, - VectorizedDictionaryEncodedParquetValuesReader reader, - int numValuesToRead, - Dictionary dict, - NullabilityHolder nullabilityHolder, - int typeWidth, - Mode mode) { - if (Mode.RLE.equals(mode)) { - reader - .intBackedDecimalDictEncodedReader() - .nextBatch(vector, idx, numValuesToRead, dict, nullabilityHolder, typeWidth); - } else if (Mode.PACKED.equals(mode)) { - ((DecimalVector) vector).set(idx, dict.decodeToInt(reader.readInteger())); - } - } - } - - class LongBackedDecimalReader extends BaseReader { - @Override - protected void nextVal( - FieldVector vector, - int idx, - ValuesAsBytesReader valuesReader, - int typeWidth, - byte[] byteArray) { - ((DecimalVector) vector).set(idx, valuesReader.getBuffer(Long.BYTES).getLong()); - } - - @Override - protected void nextDictEncodedVal( - FieldVector vector, - int idx, - VectorizedDictionaryEncodedParquetValuesReader reader, - int numValuesToRead, - Dictionary dict, - NullabilityHolder nullabilityHolder, - int typeWidth, - Mode mode) { - if (Mode.RLE.equals(mode)) { - reader - .longBackedDecimalDictEncodedReader() - .nextBatch(vector, idx, numValuesToRead, dict, nullabilityHolder, typeWidth); - } else if (Mode.PACKED.equals(mode)) { - ((DecimalVector) vector).set(idx, dict.decodeToLong(reader.readInteger())); - } - } - } - class BooleanReader extends BaseReader { @Override protected void nextVal( @@ -828,14 +736,14 @@ TimestampInt96Reader timestampInt96Reader() { return new TimestampInt96Reader(); } + /** + * @deprecated since 1.7.0, will be removed in 1.8.0. + */ + @Deprecated FixedWidthBinaryReader fixedWidthBinaryReader() { return new FixedWidthBinaryReader(); } - FixedLengthDecimalReader fixedLengthDecimalReader() { - return new FixedLengthDecimalReader(); - } - FixedSizeBinaryReader fixedSizeBinaryReader() { return new FixedSizeBinaryReader(); } @@ -844,14 +752,6 @@ VarWidthReader varWidthReader() { return new VarWidthReader(); } - IntBackedDecimalReader intBackedDecimalReader() { - return new IntBackedDecimalReader(); - } - - LongBackedDecimalReader longBackedDecimalReader() { - return new LongBackedDecimalReader(); - } - BooleanReader booleanReader() { return new BooleanReader(); } diff --git a/aws-bundle/LICENSE b/aws-bundle/LICENSE index a4ca14bf8f3e..2e7bc0c946ff 100644 --- a/aws-bundle/LICENSE +++ b/aws-bundle/LICENSE @@ -207,7 +207,7 @@ This binary artifact contains code from the following projects: -------------------------------------------------------------------------------- -Group: commons-codec Name: commons-codec Version: 1.15 +Group: commons-codec Name: commons-codec Version: 1.17.1 Project URL: https://commons.apache.org/proper/commons-codec/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt @@ -219,61 +219,61 @@ License: The Apache Software License, Version 2.0 - http://www.apache.org/licens -------------------------------------------------------------------------------- -Group: io.netty Name: netty-buffer Version: 4.1.86.Final +Group: io.netty Name: netty-buffer Version: 4.1.112.Final Project URL: https://netty.io/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 -------------------------------------------------------------------------------- -Group: io.netty Name: netty-codec Version: 4.1.86.Final +Group: io.netty Name: netty-codec Version: 4.1.112.Final Project URL: https://netty.io/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 -------------------------------------------------------------------------------- -Group: io.netty Name: netty-codec-http Version: 4.1.86.Final +Group: io.netty Name: netty-codec-http Version: 4.1.112.Final Project URL: https://netty.io/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 -------------------------------------------------------------------------------- -Group: io.netty Name: netty-codec-http2 Version: 4.1.86.Final +Group: io.netty Name: netty-codec-http2 Version: 4.1.112.Final Project URL: https://netty.io/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 -------------------------------------------------------------------------------- -Group: io.netty Name: netty-common Version: 4.1.86.Final +Group: io.netty Name: netty-common Version: 4.1.112.Final Project URL: https://netty.io/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 -------------------------------------------------------------------------------- -Group: io.netty Name: netty-handler Version: 4.1.86.Final +Group: io.netty Name: netty-handler Version: 4.1.112.Final Project URL: https://netty.io/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 -------------------------------------------------------------------------------- -Group: io.netty Name: netty-resolver Version: 4.1.86.Final +Group: io.netty Name: netty-resolver Version: 4.1.112.Final Project URL: https://netty.io/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 -------------------------------------------------------------------------------- -Group: io.netty Name: netty-transport Version: 4.1.86.Final +Group: io.netty Name: netty-transport Version: 4.1.112.Final Project URL: https://netty.io/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 -------------------------------------------------------------------------------- -Group: io.netty Name: netty-transport-classes-epoll Version: 4.1.86.Final +Group: io.netty Name: netty-transport-classes-epoll Version: 4.1.112.Final Project URL: https://netty.io/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 -------------------------------------------------------------------------------- -Group: io.netty Name: netty-transport-native-unix-common Version: 4.1.86.Final +Group: io.netty Name: netty-transport-native-unix-common Version: 4.1.112.Final Project URL: https://netty.io/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 @@ -285,13 +285,13 @@ License: Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2. -------------------------------------------------------------------------------- -Group: org.apache.httpcomponents Name: httpcore Version: 4.4.13 +Group: org.apache.httpcomponents Name: httpcore Version: 4.4.16 Project URL: http://hc.apache.org/httpcomponents-core-ga License: Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt -------------------------------------------------------------------------------- -Group: org.reactivestreams Name: reactive-streams Version: 1.0.3 +Group: org.reactivestreams Name: reactive-streams Version: 1.0.4 Project URL: http://reactive-streams.org License: CC0 - http://creativecommons.org/publicdomain/zero/1.0/ @@ -303,160 +303,220 @@ License: MIT License - http://www.opensource.org/licenses/mit-license.php -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: annotations Version: 2.20.131 +Group: software.amazon.awssdk Name: annotations Version: 2.28.5 License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: apache-client Version: 2.20.131 +Group: software.amazon.awssdk Name: apache-client Version: 2.28.5 License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: arns Version: 2.20.131 +Group: software.amazon.awssdk Name: arns Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: auth Version: 2.20.131 +Group: software.amazon.awssdk Name: auth Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: aws-core Version: 2.20.131 +Group: software.amazon.awssdk Name: aws-core Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: aws-json-protocol Version: 2.20.131 +Group: software.amazon.awssdk Name: aws-json-protocol Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: aws-query-protocol Version: 2.20.131 +Group: software.amazon.awssdk Name: aws-query-protocol Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: aws-xml-protocol Version: 2.20.131 +Group: software.amazon.awssdk Name: aws-xml-protocol Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: dynamodb Version: 2.20.131 +Group: software.amazon.awssdk Name: checksums Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: endpoints-spi Version: 2.20.131 +Group: software.amazon.awssdk Name: checksums-spi Version: 2.28.5 +Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: glue Version: 2.20.131 +Group: software.amazon.awssdk Name: crt-core Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: http-client-spi Version: 2.20.131 +Group: software.amazon.awssdk Name: dynamodb Version: 2.28.5 +Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: iam Version: 2.20.131 +Group: software.amazon.awssdk Name: endpoints-spi Version: 2.28.5 +License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: glue Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: json-utils Version: 2.20.131 +Group: software.amazon.awssdk Name: http-auth Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: kms Version: 2.20.131 +Group: software.amazon.awssdk Name: http-auth-aws Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: lakeformation Version: 2.20.131 +Group: software.amazon.awssdk Name: http-auth-aws-crt Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: metrics-spi Version: 2.20.131 +Group: software.amazon.awssdk Name: http-auth-aws-eventstream Version: 2.28.5 +Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: netty-nio-client Version: 2.20.131 +Group: software.amazon.awssdk Name: http-auth-spi Version: 2.28.5 +Project URL: https://aws.amazon.com/sdkforjava +License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: http-client-spi Version: 2.28.5 License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: profiles Version: 2.20.131 +Group: software.amazon.awssdk Name: iam Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: protocol-core Version: 2.20.131 +Group: software.amazon.awssdk Name: identity-spi Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: regions Version: 2.20.131 +Group: software.amazon.awssdk Name: json-utils Version: 2.28.5 +Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: s3 Version: 2.20.131 +Group: software.amazon.awssdk Name: kms Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: sdk-core Version: 2.20.131 +Group: software.amazon.awssdk Name: lakeformation Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: sso Version: 2.20.131 +Group: software.amazon.awssdk Name: metrics-spi Version: 2.28.5 +License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: netty-nio-client Version: 2.28.5 +License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: profiles Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: sts Version: 2.20.131 +Group: software.amazon.awssdk Name: protocol-core Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: third-party-jackson-core Version: 2.20.131 +Group: software.amazon.awssdk Name: regions Version: 2.28.5 +License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: s3 Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: utils Version: 2.20.131 +Group: software.amazon.awssdk Name: sdk-core Version: 2.28.5 +Project URL: https://aws.amazon.com/sdkforjava +License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: sso Version: 2.28.5 +Project URL: https://aws.amazon.com/sdkforjava +License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: sts Version: 2.28.5 +Project URL: https://aws.amazon.com/sdkforjava +License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: third-party-jackson-core Version: 2.28.5 +Project URL: https://aws.amazon.com/sdkforjava +License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: utils Version: 2.28.5 License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- +Group: software.amazon.awssdk.crt Name: aws-crt Version: 0.30.6 +Project URL: https://github.com/awslabs/aws-crt-java +License: The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + Group: software.amazon.eventstream Name: eventstream Version: 1.0.1 Project URL: https://github.com/awslabs/aws-eventstream-java License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 diff --git a/aws-bundle/NOTICE b/aws-bundle/NOTICE index ed353940ba38..20f84e0efd9b 100644 --- a/aws-bundle/NOTICE +++ b/aws-bundle/NOTICE @@ -7,7 +7,7 @@ The Apache Software Foundation (http://www.apache.org/). -------------------------------------------------------------------------------- -NOTICE for Group: commons-codec Name: commons-codec Version: 1.15 +NOTICE for Group: commons-codec Name: commons-codec Version: 1.17.1 src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java contains test data from http://aspell.net/test/orig/batch0.tab. @@ -23,32 +23,43 @@ Copyright (c) 2008 Alexander Beider & Stephen P. Morse. -------------------------------------------------------------------------------- -NOTICE for Group: software.amazon.awssdk Name: annotations Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: apache-client Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: arns Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: auth Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: aws-core Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: aws-json-protocol Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: aws-query-protocol Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: aws-xml-protocol Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: dynamodb Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: endpoints-spi Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: glue Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: http-client-spi Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: iam Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: json-utils Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: kms Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: lakeformation Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: metrics-spi Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: netty-nio-client Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: profiles Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: protocol-core Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: regions Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: s3 Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: sdk-core Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: sso Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: sts Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: utils Version: 2.20.131 +NOTICE for Group: software.amazon.awssdk Name: annotations Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: apache-client Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: arns Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: auth Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: aws-core Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: aws-json-protocol Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: aws-query-protocol Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: aws-xml-protocol Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: checksums Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: checksums-spi Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: crt-core Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: dynamodb Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: endpoints-spi Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: glue Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: http-auth Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: http-auth-aws Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: http-auth-aws-crt Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: http-auth-aws-eventstream Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: http-auth-spi Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: http-client-spi Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: iam Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: identity-spi Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: json-utils Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: kms Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: lakeformation Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: metrics-spi Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: netty-nio-client Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: profiles Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: protocol-core Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: regions Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: retries Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: retries-spi Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: s3 Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: sdk-core Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: sso Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: sts Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: utils Version: 2.28.5 AWS SDK for Java 2.0 Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved. @@ -71,7 +82,7 @@ The licenses for these third party components are included in LICENSE.txt -------------------------------------------------------------------------------- -NOTICE for Group: software.amazon.awssdk Name: third-party-jackson-core Version: 2.20.18 +NOTICE for Group: software.amazon.awssdk Name: third-party-jackson-core Version: 2.28.5 # Jackson JSON processor diff --git a/aws-bundle/build.gradle b/aws-bundle/build.gradle index eea591a7b76d..82bfda83498b 100644 --- a/aws-bundle/build.gradle +++ b/aws-bundle/build.gradle @@ -19,7 +19,7 @@ project(":iceberg-aws-bundle") { - apply plugin: 'io.github.goooler.shadow' + apply plugin: 'com.gradleup.shadow' tasks.jar.dependsOn tasks.shadowJar @@ -27,6 +27,7 @@ project(":iceberg-aws-bundle") { implementation platform(libs.awssdk.bom) implementation "software.amazon.awssdk:apache-client" implementation "software.amazon.awssdk:auth" + implementation "software.amazon.awssdk:http-auth-aws-crt" implementation "software.amazon.awssdk:iam" implementation "software.amazon.awssdk:sso" implementation "software.amazon.awssdk:s3" diff --git a/aws/src/integration/java/org/apache/iceberg/aws/AwsIntegTestUtil.java b/aws/src/integration/java/org/apache/iceberg/aws/AwsIntegTestUtil.java index 7e0ca6ed10b2..6b57cfd68243 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/AwsIntegTestUtil.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/AwsIntegTestUtil.java @@ -32,9 +32,11 @@ import software.amazon.awssdk.services.s3.model.Delete; import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest; import software.amazon.awssdk.services.s3.model.ListObjectVersionsRequest; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; import software.amazon.awssdk.services.s3.model.ObjectIdentifier; import software.amazon.awssdk.services.s3.model.ObjectVersion; import software.amazon.awssdk.services.s3.paginators.ListObjectVersionsIterable; +import software.amazon.awssdk.services.s3.paginators.ListObjectsV2Iterable; import software.amazon.awssdk.services.s3control.S3ControlClient; import software.amazon.awssdk.services.s3control.model.CreateAccessPointRequest; import software.amazon.awssdk.services.s3control.model.DeleteAccessPointRequest; @@ -42,6 +44,7 @@ public class AwsIntegTestUtil { private static final Logger LOG = LoggerFactory.getLogger(AwsIntegTestUtil.class); + private static final int BATCH_DELETION_SIZE = 1000; private AwsIntegTestUtil() {} @@ -95,17 +98,27 @@ public static String testAccountId() { return System.getenv("AWS_TEST_ACCOUNT_ID"); } - public static void cleanS3Bucket(S3Client s3, String bucketName, String prefix) { + /** + * Set the environment variable AWS_TEST_MULTI_REGION_ACCESS_POINT_ALIAS for a default account to + * use for testing. Developers need to create a S3 multi region access point before running + * integration tests because creating it takes a few minutes + * + * @return The alias of S3 multi region access point route to the default S3 bucket + */ + public static String testMultiRegionAccessPointAlias() { + return System.getenv("AWS_TEST_MULTI_REGION_ACCESS_POINT_ALIAS"); + } + + public static void cleanS3GeneralPurposeBucket(S3Client s3, String bucketName, String prefix) { ListObjectVersionsIterable response = s3.listObjectVersionsPaginator( ListObjectVersionsRequest.builder().bucket(bucketName).prefix(prefix).build()); List versionsToDelete = Lists.newArrayList(); - int batchDeletionSize = 1000; response.versions().stream() .forEach( version -> { versionsToDelete.add(version); - if (versionsToDelete.size() == batchDeletionSize) { + if (versionsToDelete.size() == BATCH_DELETION_SIZE) { deleteObjectVersions(s3, bucketName, versionsToDelete); versionsToDelete.clear(); } @@ -116,6 +129,45 @@ public static void cleanS3Bucket(S3Client s3, String bucketName, String prefix) } } + /** + * Method used to clean up a S3 directory bucket which doesn't care about versions + * + * @param s3 an instance of S3Client to be used to list/delete objects + * @param bucketName name of the bucket + * @param prefix the path prefix we want to remove + */ + public static void cleanS3DirectoryBucket(S3Client s3, String bucketName, String prefix) { + String newPrefix = prefix.endsWith("/") ? prefix : prefix + "/"; + ListObjectsV2Request listRequest = + ListObjectsV2Request.builder().bucket(bucketName).prefix(newPrefix).build(); + + ListObjectsV2Iterable paginatedListResponse = s3.listObjectsV2Paginator(listRequest); + List objectsToDelete = Lists.newArrayList(); + + paginatedListResponse.contents().stream() + .forEach( + s3Object -> { + if (objectsToDelete.size() == BATCH_DELETION_SIZE) { + deleteObjects(s3, bucketName, objectsToDelete); + objectsToDelete.clear(); + } + objectsToDelete.add(ObjectIdentifier.builder().key(s3Object.key()).build()); + }); + + if (!objectsToDelete.isEmpty()) { + deleteObjects(s3, bucketName, objectsToDelete); + } + } + + private static void deleteObjects( + S3Client s3, String bucketName, List objectsToDelete) { + s3.deleteObjects( + DeleteObjectsRequest.builder() + .bucket(bucketName) + .delete(Delete.builder().objects(objectsToDelete).build()) + .build()); + } + private static void deleteObjectVersions( S3Client s3, String bucket, List objectVersions) { s3.deleteObjects( diff --git a/aws/src/integration/java/org/apache/iceberg/aws/TestAssumeRoleAwsClientFactory.java b/aws/src/integration/java/org/apache/iceberg/aws/TestAssumeRoleAwsClientFactory.java index 9845d31021d8..fc6f2da9c68f 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/TestAssumeRoleAwsClientFactory.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/TestAssumeRoleAwsClientFactory.java @@ -34,8 +34,6 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import software.amazon.awssdk.core.exception.SdkServiceException; import software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient; import software.amazon.awssdk.regions.Region; @@ -51,8 +49,6 @@ public class TestAssumeRoleAwsClientFactory { - private static final Logger LOG = LoggerFactory.getLogger(TestAssumeRoleAwsClientFactory.class); - private IamClient iam; private String roleName; private Map assumeRoleProperties; diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java index ecf589d7c07a..65e37eba4cd3 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java @@ -38,8 +38,6 @@ import org.apache.iceberg.types.Types; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import software.amazon.awssdk.services.glue.GlueClient; import software.amazon.awssdk.services.glue.model.Column; import software.amazon.awssdk.services.glue.model.GetTableRequest; @@ -52,8 +50,6 @@ @SuppressWarnings({"VisibilityModifier", "HideUtilityClassConstructor"}) public class GlueTestBase { - private static final Logger LOG = LoggerFactory.getLogger(GlueTestBase.class); - // the integration test requires the following env variables static final String TEST_BUCKET_NAME = AwsIntegTestUtil.testBucketName(); @@ -114,7 +110,7 @@ public static void beforeClass() { @AfterAll public static void afterClass() { AwsIntegTestUtil.cleanGlueCatalog(GLUE, NAMESPACES); - AwsIntegTestUtil.cleanS3Bucket(S3, TEST_BUCKET_NAME, TEST_PATH_PREFIX); + AwsIntegTestUtil.cleanS3GeneralPurposeBucket(S3, TEST_BUCKET_NAME, TEST_PATH_PREFIX); } public static String getRandomName() { @@ -185,9 +181,7 @@ public static void updateTableColumns( .owner(existingTable.owner()) .parameters(existingTable.parameters()) .storageDescriptor( - existingTable - .storageDescriptor() - .toBuilder() + existingTable.storageDescriptor().toBuilder() .columns(updatedColumns) .build()) .build()) diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogCommitFailure.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogCommitFailure.java index 42b527a03742..a1df3d21da69 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogCommitFailure.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogCommitFailure.java @@ -531,7 +531,8 @@ private int metadataFileCount(TableMetadata metadata) { new File(S3TestUtil.getKeyFromUri(metadata.metadataFileLocation())) .getParent()) .build()) - .contents().stream() + .contents() + .stream() .filter(s3Object -> s3Object.key().endsWith("metadata.json")) .count(); } diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java index 6bd6a4ad383d..6ca2fe021612 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java @@ -333,8 +333,6 @@ public void testRenameTableFailsToCreateNewTable() { public void testRenameTableFailsToDeleteOldTable() { String namespace = createNamespace(); String tableName = createTable(namespace); - TableIdentifier id = TableIdentifier.of(namespace, tableName); - Table table = glueCatalog.loadTable(id); // delete the old table metadata, so that drop old table will fail String newTableName = tableName + "_2"; GLUE.updateTable( diff --git a/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/LakeFormationTestBase.java b/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/LakeFormationTestBase.java index 630db7a5a1b4..5e34c200cf5e 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/LakeFormationTestBase.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/LakeFormationTestBase.java @@ -78,7 +78,6 @@ import software.amazon.awssdk.services.lakeformation.model.GrantPermissionsRequest; import software.amazon.awssdk.services.lakeformation.model.Permission; import software.amazon.awssdk.services.lakeformation.model.PutDataLakeSettingsRequest; -import software.amazon.awssdk.services.lakeformation.model.PutDataLakeSettingsResponse; import software.amazon.awssdk.services.lakeformation.model.RegisterResourceRequest; import software.amazon.awssdk.services.lakeformation.model.Resource; import software.amazon.awssdk.services.lakeformation.model.TableResource; @@ -217,10 +216,9 @@ public static void beforeClass() throws Exception { // put lf data lake settings GetDataLakeSettingsResponse getDataLakeSettingsResponse = lakeformation.getDataLakeSettings(GetDataLakeSettingsRequest.builder().build()); - PutDataLakeSettingsResponse putDataLakeSettingsResponse = - lakeformation.putDataLakeSettings( - putDataLakeSettingsRequest( - lfRegisterPathRoleArn, getDataLakeSettingsResponse.dataLakeSettings(), true)); + lakeformation.putDataLakeSettings( + putDataLakeSettingsRequest( + lfRegisterPathRoleArn, getDataLakeSettingsResponse.dataLakeSettings(), true)); // Build test glueCatalog with lfPrivilegedRole glueCatalogPrivilegedRole = new GlueCatalog(); diff --git a/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/TestLakeFormationAwsClientFactory.java b/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/TestLakeFormationAwsClientFactory.java index 609dbf1a6acc..8a3bb71ce362 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/TestLakeFormationAwsClientFactory.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/TestLakeFormationAwsClientFactory.java @@ -52,7 +52,6 @@ public class TestLakeFormationAwsClientFactory { private static final Logger LOG = LoggerFactory.getLogger(TestLakeFormationAwsClientFactory.class); - private static final int IAM_PROPAGATION_DELAY = 10000; private static final int ASSUME_ROLE_SESSION_DURATION = 3600; private IamClient iam; diff --git a/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3FileIOIntegration.java b/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3FileIOIntegration.java index cacf04891896..9d5d41438a62 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3FileIOIntegration.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3FileIOIntegration.java @@ -41,6 +41,7 @@ import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.assertj.core.api.Assumptions; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; @@ -80,6 +81,7 @@ public class TestS3FileIOIntegration { private static String crossRegionBucketName; private static String accessPointName; private static String crossRegionAccessPointName; + private static String multiRegionAccessPointAlias; private static String prefix; private static byte[] contentBytes; private static String content; @@ -106,24 +108,32 @@ public static void beforeClass() { content = new String(contentBytes, StandardCharsets.UTF_8); kmsKeyArn = kms.createKey().keyMetadata().arn(); - AwsIntegTestUtil.createAccessPoint(s3Control, accessPointName, bucketName); - AwsIntegTestUtil.createAccessPoint( - crossRegionS3Control, crossRegionAccessPointName, crossRegionBucketName); - s3.putBucketVersioning( - PutBucketVersioningRequest.builder() - .bucket(bucketName) - .versioningConfiguration( - VersioningConfiguration.builder().status(BucketVersioningStatus.ENABLED).build()) - .build()); + if (!S3URI.isS3DirectoryBucket(bucketName)) { + s3.putBucketVersioning( + PutBucketVersioningRequest.builder() + .bucket(bucketName) + .versioningConfiguration( + VersioningConfiguration.builder().status(BucketVersioningStatus.ENABLED).build()) + .build()); + AwsIntegTestUtil.createAccessPoint(s3Control, accessPointName, bucketName); + AwsIntegTestUtil.createAccessPoint( + crossRegionS3Control, crossRegionAccessPointName, crossRegionBucketName); + multiRegionAccessPointAlias = AwsIntegTestUtil.testMultiRegionAccessPointAlias(); + } } @AfterAll public static void afterClass() { - AwsIntegTestUtil.cleanS3Bucket(s3, bucketName, prefix); - AwsIntegTestUtil.deleteAccessPoint(s3Control, accessPointName); - AwsIntegTestUtil.deleteAccessPoint(crossRegionS3Control, crossRegionAccessPointName); - kms.scheduleKeyDeletion( - ScheduleKeyDeletionRequest.builder().keyId(kmsKeyArn).pendingWindowInDays(7).build()); + if (S3URI.isS3DirectoryBucket(bucketName)) { + S3FileIO s3FileIO = new S3FileIO(clientFactory::s3); + AwsIntegTestUtil.cleanS3DirectoryBucket(s3FileIO.client(), bucketName, prefix); + } else { + AwsIntegTestUtil.cleanS3GeneralPurposeBucket(s3, bucketName, prefix); + AwsIntegTestUtil.deleteAccessPoint(s3Control, accessPointName); + AwsIntegTestUtil.deleteAccessPoint(crossRegionS3Control, crossRegionAccessPointName); + kms.scheduleKeyDeletion( + ScheduleKeyDeletionRequest.builder().keyId(kmsKeyArn).pendingWindowInDays(7).build()); + } } @BeforeEach @@ -168,6 +178,7 @@ public void testS3FileIOWithDefaultAwsClientFactoryImpl() throws Exception { @Test public void testNewInputStreamWithAccessPoint() throws Exception { + requireAccessPointSupport(); s3.putObject( PutObjectRequest.builder().bucket(bucketName).key(objectKey).build(), RequestBody.fromBytes(contentBytes)); @@ -179,8 +190,33 @@ public void testNewInputStreamWithAccessPoint() throws Exception { validateRead(s3FileIO); } + @Test + public void testCrossRegionAccessEnabled() throws Exception { + clientFactory.initialize( + ImmutableMap.of(S3FileIOProperties.CROSS_REGION_ACCESS_ENABLED, "true")); + S3Client s3Client = clientFactory.s3(); + String crossBucketObjectKey = String.format("%s/%s", prefix, UUID.randomUUID()); + String crossBucketObjectUri = + String.format("s3://%s/%s", crossRegionBucketName, crossBucketObjectKey); + try { + s3Client.putObject( + PutObjectRequest.builder() + .bucket(crossRegionBucketName) + .key(crossBucketObjectKey) + .build(), + RequestBody.fromBytes(contentBytes)); + // make a copy in cross-region bucket + S3FileIO s3FileIO = new S3FileIO(clientFactory::s3); + validateRead(s3FileIO, crossBucketObjectUri); + } finally { + AwsIntegTestUtil.cleanS3GeneralPurposeBucket( + s3Client, crossRegionBucketName, crossBucketObjectKey); + } + } + @Test public void testNewInputStreamWithCrossRegionAccessPoint() throws Exception { + requireAccessPointSupport(); clientFactory.initialize(ImmutableMap.of(S3FileIOProperties.USE_ARN_REGION_ENABLED, "true")); S3Client s3Client = clientFactory.s3(); s3Client.putObject( @@ -202,34 +238,53 @@ public void testNewInputStreamWithCrossRegionAccessPoint() throws Exception { validateRead(s3FileIO); } + @Test + public void testNewInputStreamWithMultiRegionAccessPoint() throws Exception { + Assumptions.assumeThat(multiRegionAccessPointAlias).isNotEmpty(); + clientFactory.initialize(ImmutableMap.of(S3FileIOProperties.USE_ARN_REGION_ENABLED, "true")); + S3Client s3Client = clientFactory.s3(); + s3Client.putObject( + PutObjectRequest.builder().bucket(bucketName).key(objectKey).build(), + RequestBody.fromBytes(contentBytes)); + S3FileIO s3FileIO = new S3FileIO(clientFactory::s3); + s3FileIO.initialize( + ImmutableMap.of( + S3FileIOProperties.ACCESS_POINTS_PREFIX + bucketName, + testMultiRegionAccessPointARN( + AwsIntegTestUtil.testRegion(), multiRegionAccessPointAlias))); + validateRead(s3FileIO); + } + @Test public void testNewOutputStream() throws Exception { S3FileIO s3FileIO = new S3FileIO(clientFactory::s3); write(s3FileIO); - InputStream stream = - s3.getObject(GetObjectRequest.builder().bucket(bucketName).key(objectKey).build()); - String result = IoUtils.toUtf8String(stream); - stream.close(); - assertThat(result).isEqualTo(content); + try (InputStream stream = + s3.getObject(GetObjectRequest.builder().bucket(bucketName).key(objectKey).build())) { + String result = IoUtils.toUtf8String(stream); + assertThat(result).isEqualTo(content); + } } @Test public void testNewOutputStreamWithAccessPoint() throws Exception { + requireAccessPointSupport(); S3FileIO s3FileIO = new S3FileIO(clientFactory::s3); s3FileIO.initialize( ImmutableMap.of( S3FileIOProperties.ACCESS_POINTS_PREFIX + bucketName, testAccessPointARN(AwsIntegTestUtil.testRegion(), accessPointName))); write(s3FileIO); - InputStream stream = - s3.getObject(GetObjectRequest.builder().bucket(bucketName).key(objectKey).build()); - String result = IoUtils.toUtf8String(stream); - stream.close(); - assertThat(result).isEqualTo(content); + try (InputStream stream = + s3.getObject(GetObjectRequest.builder().bucket(bucketName).key(objectKey).build())) { + String result = IoUtils.toUtf8String(stream); + assertThat(result).isEqualTo(content); + } } @Test public void testNewOutputStreamWithCrossRegionAccessPoint() throws Exception { + requireAccessPointSupport(); clientFactory.initialize(ImmutableMap.of(S3FileIOProperties.USE_ARN_REGION_ENABLED, "true")); S3Client s3Client = clientFactory.s3(); S3FileIO s3FileIO = new S3FileIO(clientFactory::s3); @@ -238,17 +293,35 @@ public void testNewOutputStreamWithCrossRegionAccessPoint() throws Exception { S3FileIOProperties.ACCESS_POINTS_PREFIX + bucketName, testAccessPointARN(AwsIntegTestUtil.testCrossRegion(), crossRegionAccessPointName))); write(s3FileIO); - InputStream stream = + try (InputStream stream = s3Client.getObject( GetObjectRequest.builder() .bucket( testAccessPointARN( AwsIntegTestUtil.testCrossRegion(), crossRegionAccessPointName)) .key(objectKey) - .build()); - String result = IoUtils.toUtf8String(stream); - stream.close(); - assertThat(result).isEqualTo(content); + .build())) { + String result = IoUtils.toUtf8String(stream); + assertThat(result).isEqualTo(content); + } + } + + @Test + public void testNewOutputStreamWithMultiRegionAccessPoint() throws Exception { + Assumptions.assumeThat(multiRegionAccessPointAlias).isNotEmpty(); + clientFactory.initialize(ImmutableMap.of(S3FileIOProperties.USE_ARN_REGION_ENABLED, "true")); + S3FileIO s3FileIO = new S3FileIO(clientFactory::s3); + s3FileIO.initialize( + ImmutableMap.of( + S3FileIOProperties.ACCESS_POINTS_PREFIX + bucketName, + testMultiRegionAccessPointARN( + AwsIntegTestUtil.testRegion(), multiRegionAccessPointAlias))); + write(s3FileIO); + try (InputStream stream = + s3.getObject(GetObjectRequest.builder().bucket(bucketName).key(objectKey).build())) { + String result = IoUtils.toUtf8String(stream); + assertThat(result).isEqualTo(content); + } } @Test @@ -266,6 +339,7 @@ public void testServerSideS3Encryption() throws Exception { @Test public void testServerSideKmsEncryption() throws Exception { + requireKMSEncryptionSupport(); S3FileIOProperties properties = new S3FileIOProperties(); properties.setSseType(S3FileIOProperties.SSE_TYPE_KMS); properties.setSseKey(kmsKeyArn); @@ -281,6 +355,7 @@ public void testServerSideKmsEncryption() throws Exception { @Test public void testServerSideKmsEncryptionWithDefaultKey() throws Exception { + requireKMSEncryptionSupport(); S3FileIOProperties properties = new S3FileIOProperties(); properties.setSseType(S3FileIOProperties.SSE_TYPE_KMS); S3FileIO s3FileIO = new S3FileIO(clientFactory::s3, properties); @@ -302,6 +377,7 @@ public void testServerSideKmsEncryptionWithDefaultKey() throws Exception { @Test public void testDualLayerServerSideKmsEncryption() throws Exception { + requireKMSEncryptionSupport(); S3FileIOProperties properties = new S3FileIOProperties(); properties.setSseType(S3FileIOProperties.DSSE_TYPE_KMS); properties.setSseKey(kmsKeyArn); @@ -317,6 +393,7 @@ public void testDualLayerServerSideKmsEncryption() throws Exception { @Test public void testServerSideCustomEncryption() throws Exception { + requireKMSEncryptionSupport(); // generate key KeyGenerator keyGenerator = KeyGenerator.getInstance("AES"); keyGenerator.init(256, new SecureRandom()); @@ -352,6 +429,7 @@ public void testServerSideCustomEncryption() throws Exception { @Test public void testACL() throws Exception { + requireACLSupport(); S3FileIOProperties properties = new S3FileIOProperties(); properties.setAcl(ObjectCannedACL.BUCKET_OWNER_FULL_CONTROL); S3FileIO s3FileIO = new S3FileIO(clientFactory::s3, properties); @@ -383,6 +461,7 @@ public void testDeleteFilesMultipleBatches() throws Exception { @Test public void testDeleteFilesMultipleBatchesWithAccessPoints() throws Exception { + requireAccessPointSupport(); S3FileIO s3FileIO = new S3FileIO(clientFactory::s3, getDeletionTestProperties()); s3FileIO.initialize( ImmutableMap.of( @@ -393,6 +472,7 @@ public void testDeleteFilesMultipleBatchesWithAccessPoints() throws Exception { @Test public void testDeleteFilesMultipleBatchesWithCrossRegionAccessPoints() throws Exception { + requireKMSEncryptionSupport(); clientFactory.initialize(ImmutableMap.of(S3FileIOProperties.USE_ARN_REGION_ENABLED, "true")); S3FileIO s3FileIO = new S3FileIO(clientFactory::s3, getDeletionTestProperties()); s3FileIO.initialize( @@ -421,8 +501,7 @@ public void testPrefixList() { List scaleSizes = Lists.newArrayList(1, 1000, 2500); String listPrefix = String.format("s3://%s/%s/%s", bucketName, prefix, "prefix-list-test"); - scaleSizes - .parallelStream() + scaleSizes.parallelStream() .forEach( scale -> { String scalePrefix = String.format("%s/%s/", listPrefix, scale); @@ -443,8 +522,7 @@ public void testPrefixDelete() { String deletePrefix = String.format("s3://%s/%s/%s", bucketName, prefix, "prefix-delete-test"); List scaleSizes = Lists.newArrayList(0, 5, 1000, 2500); - scaleSizes - .parallelStream() + scaleSizes.parallelStream() .forEach( scale -> { String scalePrefix = String.format("%s/%s/", deletePrefix, scale); @@ -456,6 +534,7 @@ public void testPrefixDelete() { @Test public void testFileRecoveryHappyPath() throws Exception { + requireVersioningSupport(); S3FileIO s3FileIO = new S3FileIO(clientFactory::s3, new S3FileIOProperties()); String filePath = String.format("s3://%s/%s/%s", bucketName, prefix, "someFile.parquet"); write(s3FileIO, filePath); @@ -468,6 +547,7 @@ public void testFileRecoveryHappyPath() throws Exception { @Test public void testFileRecoveryFailsToRecover() throws Exception { + requireVersioningSupport(); S3FileIO s3FileIO = new S3FileIO(clientFactory::s3, new S3FileIOProperties()); s3.putBucketVersioning( PutBucketVersioningRequest.builder() @@ -508,18 +588,22 @@ private void write(S3FileIO s3FileIO) throws Exception { private void write(S3FileIO s3FileIO, String uri) throws Exception { OutputFile outputFile = s3FileIO.newOutputFile(uri); - OutputStream outputStream = outputFile.create(); - IoUtils.copy(new ByteArrayInputStream(contentBytes), outputStream); - outputStream.close(); + try (OutputStream outputStream = outputFile.create()) { + IoUtils.copy(new ByteArrayInputStream(contentBytes), outputStream); + } } private void validateRead(S3FileIO s3FileIO) throws Exception { - InputFile file = s3FileIO.newInputFile(objectUri); + validateRead(s3FileIO, objectUri); + } + + private void validateRead(S3FileIO s3FileIO, String s3Uri) throws Exception { + InputFile file = s3FileIO.newInputFile(s3Uri); assertThat(file.getLength()).isEqualTo(contentBytes.length); - InputStream stream = file.newStream(); - String result = IoUtils.toUtf8String(stream); - stream.close(); - assertThat(result).isEqualTo(content); + try (InputStream stream = file.newStream()) { + String result = IoUtils.toUtf8String(stream); + assertThat(result).isEqualTo(content); + } } private String testAccessPointARN(String region, String accessPoint) { @@ -532,6 +616,13 @@ private String testAccessPointARN(String region, String accessPoint) { accessPoint); } + private String testMultiRegionAccessPointARN(String region, String alias) { + // format: arn:aws:s3::account-id:accesspoint/MultiRegionAccessPoint_alias + return String.format( + "arn:%s:s3::%s:accesspoint/%s", + PartitionMetadata.of(Region.of(region)).id(), AwsIntegTestUtil.testAccountId(), alias); + } + private void createRandomObjects(String objectPrefix, int count) { S3URI s3URI = new S3URI(objectPrefix); random @@ -543,4 +634,24 @@ private void createRandomObjects(String objectPrefix, int count) { builder -> builder.bucket(s3URI.bucket()).key(s3URI.key() + i).build(), RequestBody.empty())); } + + /** S3 Express doesn't support access points */ + private void requireAccessPointSupport() { + Assumptions.assumeThat(S3URI.isS3DirectoryBucket(bucketName)).isFalse(); + } + + /** S3 Express doesn’t support KMS/custom encryption */ + private void requireKMSEncryptionSupport() { + Assumptions.assumeThat(S3URI.isS3DirectoryBucket(bucketName)).isFalse(); + } + + /** S3 Express doesn't support versioning */ + private void requireVersioningSupport() { + Assumptions.assumeThat(S3URI.isS3DirectoryBucket(bucketName)).isFalse(); + } + + /** File ACLs aren’t supported by S3 Express */ + private void requireACLSupport() { + Assumptions.assumeThat(S3URI.isS3DirectoryBucket(bucketName)).isFalse(); + } } diff --git a/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3MultipartUpload.java b/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3MultipartUpload.java index 29d4c48927fc..901e9933b1fd 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3MultipartUpload.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3MultipartUpload.java @@ -59,7 +59,7 @@ public static void beforeClass() { @AfterAll public static void afterClass() { - AwsIntegTestUtil.cleanS3Bucket(s3, bucketName, prefix); + AwsIntegTestUtil.cleanS3GeneralPurposeBucket(s3, bucketName, prefix); } @BeforeEach diff --git a/aws/src/main/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactory.java b/aws/src/main/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactory.java index d9ea511f9bcd..4c3c305d4bae 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactory.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactory.java @@ -47,6 +47,7 @@ public S3Client s3() { .applyMutation(s3FileIOProperties::applyEndpointConfigurations) .applyMutation(s3FileIOProperties::applyServiceConfigurations) .applyMutation(s3FileIOProperties::applySignerConfiguration) + .applyMutation(s3FileIOProperties::applyRetryConfigurations) .build(); } diff --git a/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java b/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java index 81c7bd6b4bab..5974e21209e3 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java @@ -114,6 +114,7 @@ public S3Client s3() { .applyMutation(s3FileIOProperties::applySignerConfiguration) .applyMutation(s3FileIOProperties::applyS3AccessGrantsConfigurations) .applyMutation(s3FileIOProperties::applyUserAgentConfigurations) + .applyMutation(s3FileIOProperties::applyRetryConfigurations) .build(); } diff --git a/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java b/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java index 5c3afc28a98b..1a8db990578a 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java @@ -31,8 +31,6 @@ import org.apache.iceberg.relocated.com.google.common.base.Strings; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.PropertyUtil; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; import software.amazon.awssdk.auth.credentials.AwsSessionCredentials; @@ -46,8 +44,6 @@ public class AwsProperties implements Serializable { - private static final Logger LOG = LoggerFactory.getLogger(AwsProperties.class); - /** * The ID of the Glue Data Catalog where the tables reside. If none is provided, Glue * automatically uses the caller's AWS account ID by default. @@ -210,7 +206,6 @@ public class AwsProperties implements Serializable { */ public static final String REST_SESSION_TOKEN = "rest.session-token"; - private static final String HTTP_CLIENT_PREFIX = "http-client."; private final Set stsClientAssumeRoleTags; private final String clientAssumeRoleArn; diff --git a/aws/src/main/java/org/apache/iceberg/aws/HttpClientProperties.java b/aws/src/main/java/org/apache/iceberg/aws/HttpClientProperties.java index 3e615cad7024..438ae5bb0431 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/HttpClientProperties.java +++ b/aws/src/main/java/org/apache/iceberg/aws/HttpClientProperties.java @@ -44,6 +44,7 @@ public class HttpClientProperties implements Serializable { public static final String CLIENT_TYPE_APACHE = "apache"; private static final String CLIENT_PREFIX = "http-client."; + /** * If this is set under {@link #CLIENT_TYPE}, {@link * software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient} will be used as the HTTP @@ -59,6 +60,7 @@ public class HttpClientProperties implements Serializable { * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder} */ public static final String PROXY_ENDPOINT = "http-client.proxy-endpoint"; + /** * Used to configure the connection timeout in milliseconds for {@link * software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient.Builder}. This flag only @@ -69,6 +71,7 @@ public class HttpClientProperties implements Serializable { */ public static final String URLCONNECTION_CONNECTION_TIMEOUT_MS = "http-client.urlconnection.connection-timeout-ms"; + /** * Used to configure the socket timeout in milliseconds for {@link * software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient.Builder}. This flag only @@ -79,6 +82,7 @@ public class HttpClientProperties implements Serializable { */ public static final String URLCONNECTION_SOCKET_TIMEOUT_MS = "http-client.urlconnection.socket-timeout-ms"; + /** * Used to configure the connection timeout in milliseconds for {@link * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link @@ -89,6 +93,7 @@ public class HttpClientProperties implements Serializable { */ public static final String APACHE_CONNECTION_TIMEOUT_MS = "http-client.apache.connection-timeout-ms"; + /** * Used to configure the socket timeout in milliseconds for {@link * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link @@ -98,6 +103,7 @@ public class HttpClientProperties implements Serializable { * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html */ public static final String APACHE_SOCKET_TIMEOUT_MS = "http-client.apache.socket-timeout-ms"; + /** * Used to configure the connection acquisition timeout in milliseconds for {@link * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link @@ -108,6 +114,7 @@ public class HttpClientProperties implements Serializable { */ public static final String APACHE_CONNECTION_ACQUISITION_TIMEOUT_MS = "http-client.apache.connection-acquisition-timeout-ms"; + /** * Used to configure the connection max idle time in milliseconds for {@link * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link @@ -118,6 +125,7 @@ public class HttpClientProperties implements Serializable { */ public static final String APACHE_CONNECTION_MAX_IDLE_TIME_MS = "http-client.apache.connection-max-idle-time-ms"; + /** * Used to configure the connection time to live in milliseconds for {@link * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link @@ -128,6 +136,7 @@ public class HttpClientProperties implements Serializable { */ public static final String APACHE_CONNECTION_TIME_TO_LIVE_MS = "http-client.apache.connection-time-to-live-ms"; + /** * Used to configure whether to enable the expect continue setting for {@link * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link @@ -140,6 +149,7 @@ public class HttpClientProperties implements Serializable { */ public static final String APACHE_EXPECT_CONTINUE_ENABLED = "http-client.apache.expect-continue-enabled"; + /** * Used to configure the max connections number for {@link * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link @@ -149,6 +159,7 @@ public class HttpClientProperties implements Serializable { * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html */ public static final String APACHE_MAX_CONNECTIONS = "http-client.apache.max-connections"; + /** * Used to configure whether to enable the tcp keep alive setting for {@link * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link @@ -161,6 +172,7 @@ public class HttpClientProperties implements Serializable { */ public static final String APACHE_TCP_KEEP_ALIVE_ENABLED = "http-client.apache.tcp-keep-alive-enabled"; + /** * Used to configure whether to use idle connection reaper for {@link * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link diff --git a/aws/src/main/java/org/apache/iceberg/aws/lakeformation/LakeFormationAwsClientFactory.java b/aws/src/main/java/org/apache/iceberg/aws/lakeformation/LakeFormationAwsClientFactory.java index 552da4bc949c..5d37470066d2 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/lakeformation/LakeFormationAwsClientFactory.java +++ b/aws/src/main/java/org/apache/iceberg/aws/lakeformation/LakeFormationAwsClientFactory.java @@ -81,6 +81,7 @@ public S3Client s3() { .applyMutation(httpClientProperties()::applyHttpClientConfigurations) .applyMutation(s3FileIOProperties()::applyEndpointConfigurations) .applyMutation(s3FileIOProperties()::applyServiceConfigurations) + .applyMutation(s3FileIOProperties()::applyRetryConfigurations) .credentialsProvider( new LakeFormationCredentialsProvider(lakeFormation(), buildTableArn())) .region(Region.of(region())) diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/DefaultS3FileIOAwsClientFactory.java b/aws/src/main/java/org/apache/iceberg/aws/s3/DefaultS3FileIOAwsClientFactory.java index 18b40000a91a..8687d737a5d7 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/s3/DefaultS3FileIOAwsClientFactory.java +++ b/aws/src/main/java/org/apache/iceberg/aws/s3/DefaultS3FileIOAwsClientFactory.java @@ -55,6 +55,7 @@ public S3Client s3() { .applyMutation(s3FileIOProperties::applySignerConfiguration) .applyMutation(s3FileIOProperties::applyS3AccessGrantsConfigurations) .applyMutation(s3FileIOProperties::applyUserAgentConfigurations) + .applyMutation(s3FileIOProperties::applyRetryConfigurations) .build(); } } diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java b/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java index 040d3012ca7a..23b246c357c9 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java +++ b/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java @@ -297,7 +297,13 @@ private List deleteBatch(String bucket, Collection keysToDelete) @Override public Iterable listPrefix(String prefix) { - S3URI s3uri = new S3URI(prefix, s3FileIOProperties.bucketToAccessPointMapping()); + S3URI uri = new S3URI(prefix, s3FileIOProperties.bucketToAccessPointMapping()); + if (uri.useS3DirectoryBucket() + && s3FileIOProperties.isS3DirectoryBucketListPrefixAsDirectory()) { + uri = uri.toDirectoryPath(); + } + + S3URI s3uri = uri; ListObjectsV2Request request = ListObjectsV2Request.builder().bucket(s3uri.bucket()).prefix(s3uri.key()).build(); @@ -342,7 +348,7 @@ private ExecutorService executorService() { synchronized (S3FileIO.class) { if (executorService == null) { executorService = - ThreadPools.newWorkerPool( + ThreadPools.newExitingWorkerPool( "iceberg-s3fileio-delete", s3FileIOProperties.deleteThreads()); } } diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOAwsClientFactory.java b/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOAwsClientFactory.java index 411e79f58fa8..718298818a30 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOAwsClientFactory.java +++ b/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOAwsClientFactory.java @@ -29,6 +29,7 @@ public interface S3FileIOAwsClientFactory extends Serializable { * @return s3 client */ S3Client s3(); + /** * Initialize AWS client factory from catalog properties. * diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOProperties.java b/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOProperties.java index 3414c9df7046..5da758704ae5 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOProperties.java +++ b/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOProperties.java @@ -20,10 +20,12 @@ import java.io.Serializable; import java.net.URI; +import java.time.Duration; import java.util.Collections; import java.util.Map; import java.util.Set; import java.util.stream.Collectors; +import javax.xml.stream.XMLStreamException; import org.apache.iceberg.EnvironmentContext; import org.apache.iceberg.aws.AwsClientProperties; import org.apache.iceberg.aws.glue.GlueCatalog; @@ -38,6 +40,14 @@ import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider; import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; import software.amazon.awssdk.core.client.config.SdkAdvancedClientOption; +import software.amazon.awssdk.core.exception.SdkServiceException; +import software.amazon.awssdk.core.retry.RetryMode; +import software.amazon.awssdk.core.retry.RetryPolicy; +import software.amazon.awssdk.core.retry.backoff.EqualJitterBackoffStrategy; +import software.amazon.awssdk.core.retry.conditions.OrRetryCondition; +import software.amazon.awssdk.core.retry.conditions.RetryCondition; +import software.amazon.awssdk.core.retry.conditions.RetryOnExceptionsCondition; +import software.amazon.awssdk.core.retry.conditions.TokenBucketRetryCondition; import software.amazon.awssdk.services.s3.S3ClientBuilder; import software.amazon.awssdk.services.s3.S3Configuration; import software.amazon.awssdk.services.s3.model.ObjectCannedACL; @@ -283,7 +293,7 @@ public class S3FileIOProperties implements Serializable { * catalog property. After set, x-amz-storage-class header will be set to this property * *

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

Example: s3.write.storage-class=INTELLIGENT_TIERING */ @@ -366,6 +376,16 @@ public class S3FileIOProperties implements Serializable { public static final boolean DUALSTACK_ENABLED_DEFAULT = false; + /** + * Determines if S3 client will allow Cross-Region bucket access, default to false. + * + *

For more details, see + * https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/s3-cross-region.html + */ + public static final String CROSS_REGION_ACCESS_ENABLED = "s3.cross-region-access-enabled"; + + public static final boolean CROSS_REGION_ACCESS_ENABLED_DEFAULT = false; + /** * Used by {@link S3FileIO}, prefix used for bucket access point configuration. To set, we can * pass a catalog property. @@ -393,6 +413,40 @@ public class S3FileIOProperties implements Serializable { */ private static final String S3_FILE_IO_USER_AGENT = "s3fileio/" + EnvironmentContext.get(); + /** Number of times to retry S3 operations. */ + public static final String S3_RETRY_NUM_RETRIES = "s3.retry.num-retries"; + + public static final int S3_RETRY_NUM_RETRIES_DEFAULT = 5; + + /** Minimum wait time to retry a S3 operation */ + public static final String S3_RETRY_MIN_WAIT_MS = "s3.retry.min-wait-ms"; + + public static final long S3_RETRY_MIN_WAIT_MS_DEFAULT = 2_000; // 2 seconds + + /** Maximum wait time to retry a S3 read operation */ + public static final String S3_RETRY_MAX_WAIT_MS = "s3.retry.max-wait-ms"; + + public static final long S3_RETRY_MAX_WAIT_MS_DEFAULT = 20_000; // 20 seconds + + /** + * Controls whether to list prefixes as directories for S3 Directory buckets Defaults value is + * true, where it will add the "/" + * + *

Example: s3://bucket/prefix will be shown as s3://bucket/prefix/ + * + *

For more details see delimiter section in: + * https://docs.aws.amazon.com/AmazonS3/latest/API/API_ListObjectsV2.html#API_ListObjectsV2_RequestSyntax + * + *

If set to false, this will throw an error when the "/" is not provided for directory bucket. + * Turn off this feature if you are using S3FileIO.listPrefix for listing bucket prefixes that are + * not directories. This would ensure correctness and fail the operation based on S3 requirement + * when listing against a non-directory prefix in a directory bucket. + */ + public static final String S3_DIRECTORY_BUCKET_LIST_PREFIX_AS_DIRECTORY = + "s3.directory-bucket.list-prefix-as-directory"; + + public static final boolean S3_DIRECTORY_BUCKET_LIST_PREFIX_AS_DIRECTORY_DEFAULT = true; + private String sseType; private String sseKey; private String sseMd5; @@ -417,12 +471,18 @@ public class S3FileIOProperties implements Serializable { private final Map bucketToAccessPointMapping; private boolean isPreloadClientEnabled; private final boolean isDualStackEnabled; + private final boolean isCrossRegionAccessEnabled; private final boolean isPathStyleAccess; private final boolean isUseArnRegionEnabled; private final boolean isAccelerationEnabled; private final String endpoint; private final boolean isRemoteSigningEnabled; private String writeStorageClass; + private int s3RetryNumRetries; + private long s3RetryMinWaitMs; + private long s3RetryMaxWaitMs; + + private boolean s3DirectoryBucketListPrefixAsDirectory; private final Map allProperties; public S3FileIOProperties() { @@ -449,12 +509,18 @@ public S3FileIOProperties() { this.bucketToAccessPointMapping = Collections.emptyMap(); this.isPreloadClientEnabled = PRELOAD_CLIENT_ENABLED_DEFAULT; this.isDualStackEnabled = DUALSTACK_ENABLED_DEFAULT; + this.isCrossRegionAccessEnabled = CROSS_REGION_ACCESS_ENABLED_DEFAULT; this.isPathStyleAccess = PATH_STYLE_ACCESS_DEFAULT; this.isUseArnRegionEnabled = USE_ARN_REGION_ENABLED_DEFAULT; this.isAccelerationEnabled = ACCELERATION_ENABLED_DEFAULT; this.isRemoteSigningEnabled = REMOTE_SIGNING_ENABLED_DEFAULT; this.isS3AccessGrantsEnabled = S3_ACCESS_GRANTS_ENABLED_DEFAULT; this.isS3AccessGrantsFallbackToIamEnabled = S3_ACCESS_GRANTS_FALLBACK_TO_IAM_ENABLED_DEFAULT; + this.s3RetryNumRetries = S3_RETRY_NUM_RETRIES_DEFAULT; + this.s3RetryMinWaitMs = S3_RETRY_MIN_WAIT_MS_DEFAULT; + this.s3RetryMaxWaitMs = S3_RETRY_MAX_WAIT_MS_DEFAULT; + this.s3DirectoryBucketListPrefixAsDirectory = + S3_DIRECTORY_BUCKET_LIST_PREFIX_AS_DIRECTORY_DEFAULT; this.allProperties = Maps.newHashMap(); ValidationException.check( @@ -490,6 +556,9 @@ public S3FileIOProperties(Map properties) { properties, ACCELERATION_ENABLED, ACCELERATION_ENABLED_DEFAULT); this.isDualStackEnabled = PropertyUtil.propertyAsBoolean(properties, DUALSTACK_ENABLED, DUALSTACK_ENABLED_DEFAULT); + this.isCrossRegionAccessEnabled = + PropertyUtil.propertyAsBoolean( + properties, CROSS_REGION_ACCESS_ENABLED, CROSS_REGION_ACCESS_ENABLED_DEFAULT); try { this.multiPartSize = PropertyUtil.propertyAsInt(properties, MULTIPART_SIZE, MULTIPART_SIZE_DEFAULT); @@ -553,6 +622,17 @@ public S3FileIOProperties(Map properties) { properties, S3_ACCESS_GRANTS_FALLBACK_TO_IAM_ENABLED, S3_ACCESS_GRANTS_FALLBACK_TO_IAM_ENABLED_DEFAULT); + this.s3RetryNumRetries = + PropertyUtil.propertyAsInt(properties, S3_RETRY_NUM_RETRIES, S3_RETRY_NUM_RETRIES_DEFAULT); + this.s3RetryMinWaitMs = + PropertyUtil.propertyAsLong(properties, S3_RETRY_MIN_WAIT_MS, S3_RETRY_MIN_WAIT_MS_DEFAULT); + this.s3RetryMaxWaitMs = + PropertyUtil.propertyAsLong(properties, S3_RETRY_MAX_WAIT_MS, S3_RETRY_MAX_WAIT_MS_DEFAULT); + this.s3DirectoryBucketListPrefixAsDirectory = + PropertyUtil.propertyAsBoolean( + properties, + S3_DIRECTORY_BUCKET_LIST_PREFIX_AS_DIRECTORY, + S3_DIRECTORY_BUCKET_LIST_PREFIX_AS_DIRECTORY_DEFAULT); ValidationException.check( keyIdAccessKeyBothConfigured(), @@ -643,6 +723,10 @@ public boolean isDualStackEnabled() { return this.isDualStackEnabled; } + public boolean isCrossRegionAccessEnabled() { + return this.isCrossRegionAccessEnabled; + } + public boolean isPathStyleAccess() { return this.isPathStyleAccess; } @@ -753,6 +837,43 @@ public void setS3AccessGrantsFallbackToIamEnabled(boolean s3AccessGrantsFallback this.isS3AccessGrantsFallbackToIamEnabled = s3AccessGrantsFallbackToIamEnabled; } + public int s3RetryNumRetries() { + return s3RetryNumRetries; + } + + public void setS3RetryNumRetries(int s3RetryNumRetries) { + this.s3RetryNumRetries = s3RetryNumRetries; + } + + public long s3RetryMinWaitMs() { + return s3RetryMinWaitMs; + } + + public void setS3RetryMinWaitMs(long s3RetryMinWaitMs) { + this.s3RetryMinWaitMs = s3RetryMinWaitMs; + } + + public long s3RetryMaxWaitMs() { + return s3RetryMaxWaitMs; + } + + public void setS3RetryMaxWaitMs(long s3RetryMaxWaitMs) { + this.s3RetryMaxWaitMs = s3RetryMaxWaitMs; + } + + public long s3RetryTotalWaitMs() { + return (long) s3RetryNumRetries() * s3RetryMaxWaitMs(); + } + + public boolean isS3DirectoryBucketListPrefixAsDirectory() { + return s3DirectoryBucketListPrefixAsDirectory; + } + + public void setS3DirectoryBucketListPrefixAsDirectory( + boolean s3DirectoryBucketListPrefixAsDirectory) { + this.s3DirectoryBucketListPrefixAsDirectory = s3DirectoryBucketListPrefixAsDirectory; + } + private boolean keyIdAccessKeyBothConfigured() { return (accessKeyId == null) == (secretAccessKey == null); } @@ -767,7 +888,7 @@ public void applyCredentialConfigurations( /** * Configure services settings for an S3 client. The settings include: s3DualStack, - * s3UseArnRegion, s3PathStyleAccess, and s3Acceleration + * crossRegionAccessEnabled, s3UseArnRegion, s3PathStyleAccess, and s3Acceleration * *

Sample usage: * @@ -778,6 +899,7 @@ public void applyCredentialConfigurations( public void applyServiceConfigurations(T builder) { builder .dualstackEnabled(isDualStackEnabled) + .crossRegionAccessEnabled(isCrossRegionAccessEnabled) .serviceConfiguration( S3Configuration.builder() .pathStyleAccessEnabled(isPathStyleAccess) @@ -824,6 +946,71 @@ public void applyEndpointConfigurations(T builder) { } } + /** + * Override the retry configurations for an S3 client. + * + *

Sample usage: + * + *

+   *     S3Client.builder().applyMutation(s3FileIOProperties::applyRetryConfigurations)
+   * 
+ */ + public void applyRetryConfigurations(T builder) { + ClientOverrideConfiguration.Builder configBuilder = + null != builder.overrideConfiguration() + ? builder.overrideConfiguration().toBuilder() + : ClientOverrideConfiguration.builder(); + + builder.overrideConfiguration( + configBuilder + .retryPolicy( + // Use a retry strategy which will persistently retry throttled exceptions with + // exponential backoff, to give S3 a chance to autoscale. + // LEGACY mode works best here, as it will allow throttled exceptions to use all of + // the configured retry attempts. + RetryPolicy.builder(RetryMode.LEGACY) + .numRetries(s3RetryNumRetries) + .throttlingBackoffStrategy( + EqualJitterBackoffStrategy.builder() + .baseDelay(Duration.ofMillis(s3RetryMinWaitMs)) + .maxBackoffTime(Duration.ofMillis(s3RetryMaxWaitMs)) + .build()) + + // Workaround: add XMLStreamException as a retryable exception. + // https://github.com/aws/aws-sdk-java-v2/issues/5442 + // Without this workaround, we see SDK failures if there's a socket exception + // while parsing an error XML response. + .retryCondition( + OrRetryCondition.create( + RetryCondition.defaultRetryCondition(), + RetryOnExceptionsCondition.create(XMLStreamException.class))) + + // Workaround: exclude all 503s from consuming retry tokens. + // https://github.com/aws/aws-sdk-java-v2/issues/5414 + // Without this workaround, workloads which see 503s from S3 HEAD will fail + // prematurely. + .retryCapacityCondition( + TokenBucketRetryCondition.builder() + .tokenBucketSize(500) // 500 is the SDK default + .exceptionCostFunction( + e -> { + if (e instanceof SdkServiceException) { + SdkServiceException sdkServiceException = + (SdkServiceException) e; + if (sdkServiceException.isThrottlingException() + || sdkServiceException.statusCode() == 503) { + return 0; + } + } + + // 5 is the SDK default for non-throttling exceptions + return 5; + }) + .build()) + .build()) + .build()); + } + /** * Add the S3 Access Grants Plugin for an S3 client. * diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/S3InputStream.java b/aws/src/main/java/org/apache/iceberg/aws/s3/S3InputStream.java index f442a0f04a1c..4af71932e599 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/s3/S3InputStream.java +++ b/aws/src/main/java/org/apache/iceberg/aws/s3/S3InputStream.java @@ -18,9 +18,16 @@ */ package org.apache.iceberg.aws.s3; +import dev.failsafe.Failsafe; +import dev.failsafe.FailsafeException; +import dev.failsafe.RetryPolicy; import java.io.IOException; import java.io.InputStream; +import java.net.SocketException; +import java.net.SocketTimeoutException; import java.util.Arrays; +import java.util.List; +import javax.net.ssl.SSLException; import org.apache.iceberg.exceptions.NotFoundException; import org.apache.iceberg.io.FileIOMetricsContext; import org.apache.iceberg.io.IOUtil; @@ -29,8 +36,10 @@ import org.apache.iceberg.metrics.Counter; import org.apache.iceberg.metrics.MetricsContext; import org.apache.iceberg.metrics.MetricsContext.Unit; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Joiner; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.io.ByteStreams; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -43,6 +52,9 @@ class S3InputStream extends SeekableInputStream implements RangeReadable { private static final Logger LOG = LoggerFactory.getLogger(S3InputStream.class); + private static final List> RETRYABLE_EXCEPTIONS = + ImmutableList.of(SSLException.class, SocketTimeoutException.class, SocketException.class); + private final StackTraceElement[] createStack; private final S3Client s3; private final S3URI location; @@ -57,6 +69,22 @@ class S3InputStream extends SeekableInputStream implements RangeReadable { private final Counter readOperations; private int skipSize = 1024 * 1024; + private RetryPolicy retryPolicy = + RetryPolicy.builder() + .handle(RETRYABLE_EXCEPTIONS) + .onRetry( + e -> { + LOG.warn( + "Retrying read from S3, reopening stream (attempt {})", e.getAttemptCount()); + resetForRetry(); + }) + .onFailure( + e -> + LOG.error( + "Failed to read from S3 input stream after exhausting all retries", + e.getException())) + .withMaxRetries(3) + .build(); S3InputStream(S3Client s3, S3URI location) { this(s3, location, new S3FileIOProperties(), MetricsContext.nullMetrics()); @@ -92,13 +120,21 @@ public void seek(long newPos) { public int read() throws IOException { Preconditions.checkState(!closed, "Cannot read: already closed"); positionStream(); + try { + int bytesRead = Failsafe.with(retryPolicy).get(() -> stream.read()); + pos += 1; + next += 1; + readBytes.increment(); + readOperations.increment(); + + return bytesRead; + } catch (FailsafeException ex) { + if (ex.getCause() instanceof IOException) { + throw (IOException) ex.getCause(); + } - pos += 1; - next += 1; - readBytes.increment(); - readOperations.increment(); - - return stream.read(); + throw ex; + } } @Override @@ -106,13 +142,21 @@ public int read(byte[] b, int off, int len) throws IOException { Preconditions.checkState(!closed, "Cannot read: already closed"); positionStream(); - int bytesRead = stream.read(b, off, len); - pos += bytesRead; - next += bytesRead; - readBytes.increment(bytesRead); - readOperations.increment(); + try { + int bytesRead = Failsafe.with(retryPolicy).get(() -> stream.read(b, off, len)); + pos += bytesRead; + next += bytesRead; + readBytes.increment(bytesRead); + readOperations.increment(); + + return bytesRead; + } catch (FailsafeException ex) { + if (ex.getCause() instanceof IOException) { + throw (IOException) ex.getCause(); + } - return bytesRead; + throw ex; + } } @Override @@ -146,7 +190,7 @@ private InputStream readRange(String range) { public void close() throws IOException { super.close(); closed = true; - closeStream(); + closeStream(false); } private void positionStream() throws IOException { @@ -178,6 +222,10 @@ private void positionStream() throws IOException { } private void openStream() throws IOException { + openStream(false); + } + + private void openStream(boolean closeQuietly) throws IOException { GetObjectRequest.Builder requestBuilder = GetObjectRequest.builder() .bucket(location.bucket()) @@ -186,7 +234,7 @@ private void openStream() throws IOException { S3RequestUtil.configureEncryption(s3FileIOProperties, requestBuilder); - closeStream(); + closeStream(closeQuietly); try { stream = s3.getObject(requestBuilder.build(), ResponseTransformer.toInputStream()); @@ -195,7 +243,12 @@ private void openStream() throws IOException { } } - private void closeStream() throws IOException { + @VisibleForTesting + void resetForRetry() throws IOException { + openStream(true); + } + + private void closeStream(boolean closeQuietly) throws IOException { if (stream != null) { // if we aren't at the end of the stream, and the stream is abortable, then // call abort() so we don't read the remaining data with the Apache HTTP client @@ -203,6 +256,12 @@ private void closeStream() throws IOException { try { stream.close(); } catch (IOException e) { + if (closeQuietly) { + stream = null; + LOG.warn("An error occurred while closing the stream", e); + return; + } + // the Apache HTTP client will throw a ConnectionClosedException // when closing an aborted stream, which is expected if (!e.getClass().getSimpleName().equals("ConnectionClosedException")) { diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java b/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java index ef0b4debf66b..4e35c77d0557 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java +++ b/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java @@ -253,6 +253,10 @@ private void newStream() throws IOException { @Override public void close() throws IOException { + close(true); + } + + private void close(boolean completeUploads) throws IOException { if (closed) { return; } @@ -262,7 +266,9 @@ public void close() throws IOException { try { stream.close(); - completeUploads(); + if (completeUploads) { + completeUploads(); + } } finally { cleanUpStagingFiles(); } @@ -480,7 +486,7 @@ private void createStagingDirectoryIfNotExists() throws IOException, SecurityExc protected void finalize() throws Throwable { super.finalize(); if (!closed) { - close(); // releasing resources is more important than printing the warning + close(false); // releasing resources is more important than printing the warning String trace = Joiner.on("\n\t").join(Arrays.copyOfRange(createStack, 1, createStack.length)); LOG.warn("Unclosed output stream created by:\n\t{}", trace); } diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/S3URI.java b/aws/src/main/java/org/apache/iceberg/aws/s3/S3URI.java index 79b4e695defc..9cfba5fca35c 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/s3/S3URI.java +++ b/aws/src/main/java/org/apache/iceberg/aws/s3/S3URI.java @@ -37,6 +37,9 @@ class S3URI { private static final String QUERY_DELIM = "\\?"; private static final String FRAGMENT_DELIM = "#"; + /** Suffix of S3Express storage bucket names. */ + private static final String S3_DIRECTORY_BUCKET_SUFFIX = "--x-s3"; + private final String location; private final String scheme; private final String bucket; @@ -115,4 +118,34 @@ public String scheme() { public String toString() { return location; } + + /** + * Converts the current S3URI to a directory path. + * + *

This method ensures that the S3URI represents a directory by adding a "/" delimiter at the + * end of the prefix if it's not already present. + * + * @return a S3URI with the directory path configured + */ + public S3URI toDirectoryPath() { + if (key.endsWith(PATH_DELIM)) { + return this; + } + return new S3URI(String.format("%s://%s/%s/", scheme, bucket, key)); + } + + public boolean useS3DirectoryBucket() { + return isS3DirectoryBucket(this.bucket); + } + + /** + * Check if the bucket name indicates the bucket is a directory bucket. This method does not check + * against the S3 service. + * + * @param bucket bucket to probe. + * @return true if the bucket name indicates the bucket is a directory bucket + */ + public static boolean isS3DirectoryBucket(final String bucket) { + return bucket.endsWith(S3_DIRECTORY_BUCKET_SUFFIX); + } } diff --git a/aws/src/test/java/org/apache/iceberg/aws/TestS3FileIOProperties.java b/aws/src/test/java/org/apache/iceberg/aws/TestS3FileIOProperties.java index 92ed3648bcfc..58332d42588e 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/TestS3FileIOProperties.java +++ b/aws/src/test/java/org/apache/iceberg/aws/TestS3FileIOProperties.java @@ -36,6 +36,7 @@ import software.amazon.awssdk.auth.credentials.AwsSessionCredentials; import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; import software.amazon.awssdk.core.client.config.SdkAdvancedClientOption; +import software.amazon.awssdk.core.retry.RetryPolicy; import software.amazon.awssdk.core.signer.Signer; import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.S3ClientBuilder; @@ -239,7 +240,7 @@ public void testS3RemoteSigningEnabled() { } @Test - public void s3RemoteSigningEnabledWithUserAgent() { + public void s3RemoteSigningEnabledWithUserAgentAndRetryPolicy() { String uri = "http://localhost:12345"; Map properties = ImmutableMap.of( @@ -249,6 +250,7 @@ public void s3RemoteSigningEnabledWithUserAgent() { s3Properties.applySignerConfiguration(builder); s3Properties.applyUserAgentConfigurations(builder); + s3Properties.applyRetryConfigurations(builder); Optional userAgent = builder.overrideConfiguration().advancedOption(SdkAdvancedClientOption.USER_AGENT_PREFIX); @@ -260,6 +262,9 @@ public void s3RemoteSigningEnabledWithUserAgent() { S3V4RestSignerClient signerClient = (S3V4RestSignerClient) signer.get(); assertThat(signerClient.baseSignerUri()).isEqualTo(uri); assertThat(signerClient.properties()).isEqualTo(properties); + + Optional retryPolicy = builder.overrideConfiguration().retryPolicy(); + assertThat(retryPolicy).isPresent().get().isInstanceOf(RetryPolicy.class); } @Test @@ -307,4 +312,12 @@ public void testS3AccessGrantsDisabled() { s3Properties.applyS3AccessGrantsConfigurations(builder); assertThat(builder.plugins().size()).isEqualTo(0); } + + @Test + public void testIsTreatS3DirectoryBucketListPrefixAsDirectoryEnabled() { + Map map = Maps.newHashMap(); + map.put(S3FileIOProperties.S3_DIRECTORY_BUCKET_LIST_PREFIX_AS_DIRECTORY, "false"); + S3FileIOProperties properties = new S3FileIOProperties(map); + assertThat(properties.isS3DirectoryBucketListPrefixAsDirectory()).isEqualTo(false); + } } diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/MinioContainer.java b/aws/src/test/java/org/apache/iceberg/aws/s3/MinioContainer.java deleted file mode 100644 index b6a8d960981c..000000000000 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/MinioContainer.java +++ /dev/null @@ -1,68 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.aws.s3; - -import java.net.URI; -import java.time.Duration; -import org.testcontainers.containers.GenericContainer; -import org.testcontainers.containers.wait.strategy.HttpWaitStrategy; -import org.testcontainers.utility.Base58; -import software.amazon.awssdk.auth.credentials.AwsCredentials; - -public class MinioContainer extends GenericContainer { - - private static final int DEFAULT_PORT = 9000; - private static final String DEFAULT_IMAGE = "minio/minio"; - private static final String DEFAULT_TAG = "edge"; - - private static final String MINIO_ACCESS_KEY = "MINIO_ACCESS_KEY"; - private static final String MINIO_SECRET_KEY = "MINIO_SECRET_KEY"; - - private static final String DEFAULT_STORAGE_DIRECTORY = "/data"; - private static final String HEALTH_ENDPOINT = "/minio/health/ready"; - - public MinioContainer(AwsCredentials credentials) { - this(DEFAULT_IMAGE + ":" + DEFAULT_TAG, credentials); - } - - public MinioContainer(String image, AwsCredentials credentials) { - super(image == null ? DEFAULT_IMAGE + ":" + DEFAULT_TAG : image); - this.withNetworkAliases("minio-" + Base58.randomString(6)) - .withCommand("server", DEFAULT_STORAGE_DIRECTORY) - .addExposedPort(DEFAULT_PORT); - if (credentials != null) { - this.withEnv(MINIO_ACCESS_KEY, credentials.accessKeyId()) - .withEnv(MINIO_SECRET_KEY, credentials.secretAccessKey()); - } - - // this enables virtual-host-style requests. see - // https://github.com/minio/minio/tree/master/docs/config#domain - this.withEnv("MINIO_DOMAIN", "localhost"); - - setWaitStrategy( - new HttpWaitStrategy() - .forPort(DEFAULT_PORT) - .forPath(HEALTH_ENDPOINT) - .withStartupTimeout(Duration.ofMinutes(2))); - } - - public URI getURI() { - return URI.create("http://" + getHost() + ":" + getMappedPort(DEFAULT_PORT)); - } -} diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/MinioUtil.java b/aws/src/test/java/org/apache/iceberg/aws/s3/MinioUtil.java new file mode 100644 index 000000000000..ff131784a55b --- /dev/null +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/MinioUtil.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.aws.s3; + +import java.net.URI; +import org.testcontainers.containers.MinIOContainer; +import org.testcontainers.utility.DockerImageName; +import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentials; +import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.S3ClientBuilder; + +public class MinioUtil { + + private MinioUtil() {} + + public static MinIOContainer createContainer() { + return createContainer(null); + } + + public static MinIOContainer createContainer(AwsCredentials credentials) { + var container = new MinIOContainer(DockerImageName.parse("minio/minio:latest")); + + // this enables virtual-host-style requests. see + // https://github.com/minio/minio/tree/master/docs/config#domain + container.withEnv("MINIO_DOMAIN", "localhost"); + + if (credentials != null) { + container.withUserName(credentials.accessKeyId()); + container.withPassword(credentials.secretAccessKey()); + } + + return container; + } + + public static S3Client createS3Client(MinIOContainer container) { + URI uri = URI.create(container.getS3URL()); + S3ClientBuilder builder = S3Client.builder(); + builder.credentialsProvider( + StaticCredentialsProvider.create( + AwsBasicCredentials.create(container.getUserName(), container.getPassword()))); + builder.applyMutation(mutator -> mutator.endpointOverride(uri)); + builder.region(Region.US_EAST_1); + builder.forcePathStyle(true); // OSX won't resolve subdomains + return builder.build(); + } +} diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/TestFlakyS3InputStream.java b/aws/src/test/java/org/apache/iceberg/aws/s3/TestFlakyS3InputStream.java new file mode 100644 index 000000000000..f98d1a3d4471 --- /dev/null +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestFlakyS3InputStream.java @@ -0,0 +1,232 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.aws.s3; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.spy; + +import java.io.IOException; +import java.io.InputStream; +import java.net.SocketTimeoutException; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Stream; +import javax.net.ssl.SSLException; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.core.ResponseInputStream; +import software.amazon.awssdk.core.exception.SdkClientException; +import software.amazon.awssdk.core.sync.RequestBody; +import software.amazon.awssdk.core.sync.ResponseTransformer; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.CreateBucketRequest; +import software.amazon.awssdk.services.s3.model.CreateBucketResponse; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; +import software.amazon.awssdk.services.s3.model.HeadObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; +import software.amazon.awssdk.services.s3.model.PutObjectResponse; + +public class TestFlakyS3InputStream extends TestS3InputStream { + + private AtomicInteger resetForRetryCounter; + + @BeforeEach + public void setupTest() { + resetForRetryCounter = new AtomicInteger(0); + } + + @Override + S3InputStream newInputStream(S3Client s3Client, S3URI uri) { + return new S3InputStream(s3Client, uri) { + @Override + void resetForRetry() throws IOException { + resetForRetryCounter.incrementAndGet(); + super.resetForRetry(); + } + }; + } + + @ParameterizedTest + @MethodSource("retryableExceptions") + public void testReadWithFlakyStreamRetrySucceed(IOException exception) throws Exception { + testRead(flakyStreamClient(new AtomicInteger(3), exception)); + assertThat(resetForRetryCounter.get()).isEqualTo(2); + } + + @ParameterizedTest + @MethodSource("retryableExceptions") + public void testReadWithFlakyStreamExhaustedRetries(IOException exception) { + assertThatThrownBy(() -> testRead(flakyStreamClient(new AtomicInteger(5), exception))) + .isInstanceOf(exception.getClass()) + .hasMessage(exception.getMessage()); + assertThat(resetForRetryCounter.get()).isEqualTo(3); + } + + @ParameterizedTest + @MethodSource("nonRetryableExceptions") + public void testReadWithFlakyStreamNonRetryableException(IOException exception) { + assertThatThrownBy(() -> testRead(flakyStreamClient(new AtomicInteger(3), exception))) + .isInstanceOf(exception.getClass()) + .hasMessage(exception.getMessage()); + assertThat(resetForRetryCounter.get()).isEqualTo(0); + } + + @ParameterizedTest + @MethodSource("retryableExceptions") + public void testSeekWithFlakyStreamRetrySucceed(IOException exception) throws Exception { + testSeek(flakyStreamClient(new AtomicInteger(3), exception)); + assertThat(resetForRetryCounter.get()).isEqualTo(2); + } + + @ParameterizedTest + @MethodSource("retryableExceptions") + public void testSeekWithFlakyStreamExhaustedRetries(IOException exception) { + assertThatThrownBy(() -> testSeek(flakyStreamClient(new AtomicInteger(5), exception))) + .isInstanceOf(exception.getClass()) + .hasMessage(exception.getMessage()); + assertThat(resetForRetryCounter.get()).isEqualTo(3); + } + + @ParameterizedTest + @MethodSource("nonRetryableExceptions") + public void testSeekWithFlakyStreamNonRetryableException(IOException exception) { + assertThatThrownBy(() -> testSeek(flakyStreamClient(new AtomicInteger(3), exception))) + .isInstanceOf(exception.getClass()) + .hasMessage(exception.getMessage()); + assertThat(resetForRetryCounter.get()).isEqualTo(0); + } + + private static Stream retryableExceptions() { + return Stream.of( + Arguments.of( + new SocketTimeoutException("socket timeout exception"), + new SSLException("some ssl exception"))); + } + + private static Stream nonRetryableExceptions() { + return Stream.of(Arguments.of(new IOException("some generic non-retryable IO exception"))); + } + + private S3ClientWrapper flakyStreamClient(AtomicInteger counter, IOException failure) { + S3ClientWrapper flakyClient = spy(new S3ClientWrapper(s3Client())); + doAnswer(invocation -> new FlakyInputStream(invocation.callRealMethod(), counter, failure)) + .when(flakyClient) + .getObject(any(GetObjectRequest.class), any(ResponseTransformer.class)); + return flakyClient; + } + + /** Wrapper for S3 client, used to mock the final class DefaultS3Client */ + public static class S3ClientWrapper implements S3Client { + + private final S3Client delegate; + + public S3ClientWrapper(S3Client delegate) { + this.delegate = delegate; + } + + @Override + public String serviceName() { + return delegate.serviceName(); + } + + @Override + public void close() { + delegate.close(); + } + + @Override + public ReturnT getObject( + GetObjectRequest getObjectRequest, + ResponseTransformer responseTransformer) + throws AwsServiceException, SdkClientException { + return delegate.getObject(getObjectRequest, responseTransformer); + } + + @Override + public HeadObjectResponse headObject(HeadObjectRequest headObjectRequest) + throws AwsServiceException, SdkClientException { + return delegate.headObject(headObjectRequest); + } + + @Override + public PutObjectResponse putObject(PutObjectRequest putObjectRequest, RequestBody requestBody) + throws AwsServiceException, SdkClientException { + return delegate.putObject(putObjectRequest, requestBody); + } + + @Override + public CreateBucketResponse createBucket(CreateBucketRequest createBucketRequest) + throws AwsServiceException, SdkClientException { + return delegate.createBucket(createBucketRequest); + } + } + + static class FlakyInputStream extends InputStream { + private final ResponseInputStream delegate; + private final AtomicInteger counter; + private final int round; + private final IOException exception; + + FlakyInputStream(Object invocationResponse, AtomicInteger counter, IOException exception) { + this.delegate = (ResponseInputStream) invocationResponse; + this.counter = counter; + this.round = counter.get(); + this.exception = exception; + } + + private void checkCounter() throws IOException { + // for every round of n invocations, only the last call succeeds + if (counter.decrementAndGet() == 0) { + counter.set(round); + } else { + throw exception; + } + } + + @Override + public int read() throws IOException { + checkCounter(); + return delegate.read(); + } + + @Override + public int read(byte[] b) throws IOException { + checkCounter(); + return delegate.read(b); + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + checkCounter(); + return delegate.read(b, off, len); + } + + @Override + public void close() throws IOException { + delegate.close(); + } + } +} diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/TestMinioUtil.java b/aws/src/test/java/org/apache/iceberg/aws/s3/TestMinioUtil.java new file mode 100644 index 000000000000..9955aa7f8459 --- /dev/null +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestMinioUtil.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.aws.s3; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.UUID; +import org.junit.jupiter.api.Test; +import org.testcontainers.containers.MinIOContainer; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; +import software.amazon.awssdk.core.sync.RequestBody; +import software.amazon.awssdk.core.sync.ResponseTransformer; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.CreateBucketRequest; +import software.amazon.awssdk.services.s3.model.CreateBucketResponse; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; +import software.amazon.awssdk.services.s3.model.PutObjectResponse; +import software.amazon.awssdk.services.s3.model.S3Exception; + +@Testcontainers +public class TestMinioUtil { + @Container private static final MinIOContainer MINIO = MinioUtil.createContainer(); + + @Test + void validateS3ConditionalWrites() { + S3Client s3Client = MinioUtil.createS3Client(MINIO); + + String bucket = "test-bucket-" + UUID.randomUUID(); + + CreateBucketResponse createBucketResponse = + s3Client.createBucket(CreateBucketRequest.builder().bucket(bucket).build()); + assertThat(createBucketResponse.sdkHttpResponse().isSuccessful()).isTrue(); + + String key = "test-key-" + UUID.randomUUID().toString(); + for (int i = 0; i < 5; i++) { + String payload = "test-payload-" + i; + PutObjectRequest request = + PutObjectRequest.builder().bucket(bucket).key(key).ifNoneMatch("*").build(); + RequestBody body = RequestBody.fromString(payload); + if (i == 0) { + PutObjectResponse response = s3Client.putObject(request, body); + assertThat(response.sdkHttpResponse().isSuccessful()).isTrue(); + } else { + assertThatThrownBy(() -> s3Client.putObject(request, body)) + .isInstanceOf(S3Exception.class) + .hasMessageContaining("Service: S3, Status Code: 412") + .hasMessageContaining("At least one of the pre-conditions you specified did not hold"); + } + } + + var getResponse = + s3Client.getObject( + request -> request.bucket(bucket).key(key), ResponseTransformer.toBytes()); + String responseBody = getResponse.asUtf8String(); + assertThat(responseBody).isEqualTo("test-payload-0"); + } +} diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIO.java b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIO.java index 38489e367434..cda6216fe83c 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIO.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIO.java @@ -20,6 +20,8 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; import static org.mockito.AdditionalAnswers.delegatesTo; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doReturn; @@ -29,15 +31,21 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; -import com.adobe.testing.s3mock.junit5.S3MockExtension; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.nio.ByteBuffer; +import java.time.Instant; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Random; +import java.util.Spliterator; +import java.util.Spliterators; import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.BaseTable; @@ -58,6 +66,7 @@ import org.apache.iceberg.io.BulkDeletionFailureException; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.FileIOParser; +import org.apache.iceberg.io.FileInfo; import org.apache.iceberg.io.IOUtil; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; @@ -74,8 +83,10 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.jupiter.api.extension.RegisterExtension; +import org.mockito.Mockito; +import org.testcontainers.containers.MinIOContainer; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; import software.amazon.awssdk.core.sync.RequestBody; import software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient; import software.amazon.awssdk.regions.Region; @@ -86,14 +97,17 @@ import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest; import software.amazon.awssdk.services.s3.model.DeleteObjectsResponse; import software.amazon.awssdk.services.s3.model.HeadObjectRequest; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Response; import software.amazon.awssdk.services.s3.model.S3Error; +import software.amazon.awssdk.services.s3.model.S3Object; +import software.amazon.awssdk.services.s3.paginators.ListObjectsV2Iterable; -@ExtendWith(S3MockExtension.class) +@Testcontainers public class TestS3FileIO { - @RegisterExtension - public static final S3MockExtension S3_MOCK = S3MockExtension.builder().silent().build(); + @Container private static final MinIOContainer MINIO = MinioUtil.createContainer(); - public SerializableSupplier s3 = S3_MOCK::createS3ClientV2; + private final SerializableSupplier s3 = () -> MinioUtil.createS3Client(MINIO); private final S3Client s3mock = mock(S3Client.class, delegatesTo(s3.get())); private final Random random = new Random(1); private final int numBucketsForBatchDeletion = 3; @@ -101,6 +115,9 @@ public class TestS3FileIO { private final int batchDeletionSize = 5; private S3FileIO s3FileIO; + private static final String S3_GENERAL_PURPOSE_BUCKET = "bucket"; + private static final String S3_DIRECTORY_BUCKET = "directory-bucket-usw2-az1--x-s3"; + private final Map properties = ImmutableMap.of( "s3.write.tags.tagKey1", @@ -112,7 +129,7 @@ public class TestS3FileIO { public void before() { s3FileIO = new S3FileIO(() -> s3mock); s3FileIO.initialize(properties); - createBucket("bucket"); + createBucket(S3_GENERAL_PURPOSE_BUCKET); for (int i = 1; i <= numBucketsForBatchDeletion; i++) { createBucket(batchDeletionBucketPrefix + i); } @@ -243,8 +260,7 @@ public void testPrefixList() { List scaleSizes = Lists.newArrayList(1, 1000, 2500); - scaleSizes - .parallelStream() + scaleSizes.parallelStream() .forEach( scale -> { String scalePrefix = String.format("%s/%s/", prefix, scale); @@ -258,6 +274,89 @@ public void testPrefixList() { assertThat(Streams.stream(s3FileIO.listPrefix(prefix)).count()).isEqualTo(totalFiles); } + /** + * Tests that we correctly insert the backslash for s3 express buckets. Currently the Adobe S3 + * Mock doesn't cater for express buckets eg. When you call createBucket with s3 express + * configurations it still just returns a general bucket TODO Update to use S3Mock when it behaves + * as expected. + */ + @Test + public void testPrefixListWithExpressAddSlash() { + assertPrefixIsAddedCorrectly("path/to/list", properties); + + Map newProperties = + ImmutableMap.of( + "s3.write.tags.tagKey1", + "TagValue1", + "s3.delete.batch-size", + Integer.toString(batchDeletionSize), + "s3.directory-bucket.list-prefix-as-directory", + "true"); + assertPrefixIsAddedCorrectly("path/to/list/", newProperties); + } + + public void assertPrefixIsAddedCorrectly(String suffix, Map props) { + String prefix = String.format("s3://%s/%s", S3_DIRECTORY_BUCKET, suffix); + + S3Client localMockedClient = mock(S3Client.class); + + List s3Objects = + Arrays.asList( + S3Object.builder() + .key("path/to/list/file1.txt") + .size(1024L) + .lastModified(Instant.now()) + .build(), + S3Object.builder() + .key("path/to/list/file2.txt") + .size(2048L) + .lastModified(Instant.now().minusSeconds(60)) + .build()); + + ListObjectsV2Response response = ListObjectsV2Response.builder().contents(s3Objects).build(); + + ListObjectsV2Iterable mockedResponse = mock(ListObjectsV2Iterable.class); + + Mockito.when(mockedResponse.stream()).thenReturn(Stream.of(response)); + + Mockito.when( + localMockedClient.listObjectsV2Paginator( + ListObjectsV2Request.builder() + .prefix("path/to/list/") + .bucket(S3_DIRECTORY_BUCKET) + .build())) + .thenReturn(mockedResponse); + + // Initialize S3FileIO with the mocked client + S3FileIO localS3FileIo = new S3FileIO(() -> localMockedClient); + localS3FileIo.initialize(props); + + // Perform the listing + List fileInfoList = + StreamSupport.stream( + Spliterators.spliteratorUnknownSize( + localS3FileIo.listPrefix(prefix).iterator(), Spliterator.ORDERED), + false) + .collect(Collectors.toList()); + + // Assert that the returned FileInfo instances match the expected values + assertEquals(2, fileInfoList.size()); + assertTrue( + fileInfoList.stream() + .anyMatch( + fi -> + fi.location().endsWith("file1.txt") + && fi.size() == 1024 + && fi.createdAtMillis() > Instant.now().minusSeconds(120).toEpochMilli())); + assertTrue( + fileInfoList.stream() + .anyMatch( + fi -> + fi.location().endsWith("file2.txt") + && fi.size() == 2048 + && fi.createdAtMillis() < Instant.now().minusSeconds(30).toEpochMilli())); + } + /** * Ignoring because the test is flaky, failing with 500s from S3Mock. Coverage of prefix delete * exists through integration tests. @@ -384,6 +483,18 @@ public void testResolvingFileIOLoad() { assertThat(result).isInstanceOf(S3FileIO.class); } + @Test + public void testResolvingFileIOLoadWithoutConf() { + ResolvingFileIO resolvingFileIO = new ResolvingFileIO(); + resolvingFileIO.initialize(ImmutableMap.of()); + FileIO result = + DynMethods.builder("io") + .hiddenImpl(ResolvingFileIO.class, String.class) + .build(resolvingFileIO) + .invoke("s3://foo/bar"); + assertThat(result).isInstanceOf(S3FileIO.class); + } + @Test public void testInputFileWithDataFile() throws IOException { String location = "s3://bucket/path/to/data-file.parquet"; diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIOProperties.java b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIOProperties.java index f445a2d22422..71b931257cf5 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIOProperties.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIOProperties.java @@ -36,6 +36,8 @@ import org.mockito.ArgumentCaptor; import org.mockito.Mockito; import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; +import software.amazon.awssdk.core.retry.RetryPolicy; +import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.S3ClientBuilder; import software.amazon.awssdk.services.s3.S3Configuration; import software.amazon.awssdk.services.s3.model.ObjectCannedACL; @@ -70,6 +72,9 @@ public void testS3FileIOPropertiesDefaultValues() { assertThat(S3FileIOProperties.DUALSTACK_ENABLED_DEFAULT) .isEqualTo(s3FileIOProperties.isDualStackEnabled()); + assertThat(S3FileIOProperties.CROSS_REGION_ACCESS_ENABLED_DEFAULT) + .isEqualTo(s3FileIOProperties.isCrossRegionAccessEnabled()); + assertThat(S3FileIOProperties.PATH_STYLE_ACCESS_DEFAULT) .isEqualTo(s3FileIOProperties.isPathStyleAccess()); @@ -153,6 +158,11 @@ public void testS3FileIOProperties() { S3FileIOProperties.DUALSTACK_ENABLED, String.valueOf(s3FileIOProperties.isDualStackEnabled())); + assertThat(map) + .containsEntry( + S3FileIOProperties.CROSS_REGION_ACCESS_ENABLED, + String.valueOf(s3FileIOProperties.isCrossRegionAccessEnabled())); + assertThat(map) .containsEntry( S3FileIOProperties.PATH_STYLE_ACCESS, @@ -380,6 +390,7 @@ private Map getTestProperties() { map.put(S3FileIOProperties.USE_ARN_REGION_ENABLED, "true"); map.put(S3FileIOProperties.ACCELERATION_ENABLED, "true"); map.put(S3FileIOProperties.DUALSTACK_ENABLED, "true"); + map.put(S3FileIOProperties.CROSS_REGION_ACCESS_ENABLED, "true"); map.put( S3FileIOProperties.MULTIPART_SIZE, String.valueOf(S3FileIOProperties.MULTIPART_SIZE_DEFAULT)); @@ -425,6 +436,7 @@ public void testApplyCredentialConfigurations() { public void testApplyS3ServiceConfigurations() { Map properties = Maps.newHashMap(); properties.put(S3FileIOProperties.DUALSTACK_ENABLED, "true"); + properties.put(S3FileIOProperties.CROSS_REGION_ACCESS_ENABLED, "true"); properties.put(S3FileIOProperties.PATH_STYLE_ACCESS, "true"); properties.put(S3FileIOProperties.USE_ARN_REGION_ENABLED, "true"); // acceleration enabled has to be set to false if path style is true @@ -436,6 +448,7 @@ public void testApplyS3ServiceConfigurations() { ArgumentCaptor.forClass(S3Configuration.class); Mockito.doReturn(mockA).when(mockA).dualstackEnabled(Mockito.anyBoolean()); + Mockito.doReturn(mockA).when(mockA).crossRegionAccessEnabled(Mockito.anyBoolean()); Mockito.doReturn(mockA).when(mockA).serviceConfiguration(Mockito.any(S3Configuration.class)); s3FileIOProperties.applyServiceConfigurations(mockA); @@ -491,4 +504,17 @@ public void testApplyUserAgentConfigurations() { Mockito.verify(mockS3ClientBuilder) .overrideConfiguration(Mockito.any(ClientOverrideConfiguration.class)); } + + @Test + public void testApplyRetryConfiguration() { + Map properties = Maps.newHashMap(); + properties.put(S3FileIOProperties.S3_RETRY_NUM_RETRIES, "999"); + S3FileIOProperties s3FileIOProperties = new S3FileIOProperties(properties); + + S3ClientBuilder builder = S3Client.builder(); + s3FileIOProperties.applyRetryConfigurations(builder); + + RetryPolicy retryPolicy = builder.overrideConfiguration().retryPolicy().get(); + assertThat(retryPolicy.numRetries()).as("retries was not set").isEqualTo(999); + } } diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3InputStream.java b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3InputStream.java index ed71e259a26c..f8903842df37 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3InputStream.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3InputStream.java @@ -21,7 +21,6 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import com.adobe.testing.s3mock.junit5.S3MockExtension; import java.io.IOException; import java.util.Arrays; import java.util.Random; @@ -30,8 +29,9 @@ import org.apache.iceberg.io.SeekableInputStream; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.jupiter.api.extension.RegisterExtension; +import org.testcontainers.containers.MinIOContainer; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; import software.amazon.awssdk.core.sync.RequestBody; import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.model.BucketAlreadyExistsException; @@ -39,12 +39,11 @@ import software.amazon.awssdk.services.s3.model.CreateBucketRequest; import software.amazon.awssdk.services.s3.model.PutObjectRequest; -@ExtendWith(S3MockExtension.class) +@Testcontainers public class TestS3InputStream { - @RegisterExtension - public static final S3MockExtension S3_MOCK = S3MockExtension.builder().silent().build(); + @Container private static final MinIOContainer MINIO = MinioUtil.createContainer(); - private final S3Client s3 = S3_MOCK.createS3ClientV2(); + private final S3Client s3 = MinioUtil.createS3Client(MINIO); private final Random random = new Random(1); @BeforeEach @@ -54,16 +53,22 @@ public void before() { @Test public void testRead() throws Exception { + testRead(s3); + } + + S3InputStream newInputStream(S3Client s3Client, S3URI uri) { + return new S3InputStream(s3Client, uri); + } + + protected void testRead(S3Client s3Client) throws Exception { S3URI uri = new S3URI("s3://bucket/path/to/read.dat"); int dataSize = 1024 * 1024 * 10; byte[] data = randomData(dataSize); writeS3Data(uri, data); - try (SeekableInputStream in = new S3InputStream(s3, uri)) { + try (SeekableInputStream in = newInputStream(s3Client, uri)) { int readSize = 1024; - byte[] actual = new byte[readSize]; - readAndCheck(in, in.getPos(), readSize, data, false); readAndCheck(in, in.getPos(), readSize, data, true); @@ -111,6 +116,10 @@ private void readAndCheck( @Test public void testRangeRead() throws Exception { + testRangeRead(s3); + } + + protected void testRangeRead(S3Client s3Client) throws Exception { S3URI uri = new S3URI("s3://bucket/path/to/range-read.dat"); int dataSize = 1024 * 1024 * 10; byte[] expected = randomData(dataSize); @@ -122,7 +131,7 @@ public void testRangeRead() throws Exception { writeS3Data(uri, expected); - try (RangeReadable in = new S3InputStream(s3, uri)) { + try (RangeReadable in = newInputStream(s3Client, uri)) { // first 1k position = 0; offset = 0; @@ -154,7 +163,7 @@ private void readAndCheckRanges( @Test public void testClose() throws Exception { S3URI uri = new S3URI("s3://bucket/path/to/closed.dat"); - SeekableInputStream closed = new S3InputStream(s3, uri); + SeekableInputStream closed = newInputStream(s3, uri); closed.close(); assertThatThrownBy(() -> closed.seek(0)) .isInstanceOf(IllegalStateException.class) @@ -163,12 +172,16 @@ public void testClose() throws Exception { @Test public void testSeek() throws Exception { + testSeek(s3); + } + + protected void testSeek(S3Client s3Client) throws Exception { S3URI uri = new S3URI("s3://bucket/path/to/seek.dat"); byte[] expected = randomData(1024 * 1024); writeS3Data(uri, expected); - try (SeekableInputStream in = new S3InputStream(s3, uri)) { + try (SeekableInputStream in = newInputStream(s3Client, uri)) { in.seek(expected.length / 2); byte[] actual = new byte[expected.length / 2]; IOUtil.readFully(in, actual, 0, expected.length / 2); @@ -200,4 +213,8 @@ private void createBucket(String bucketName) { // don't do anything } } + + protected S3Client s3Client() { + return s3; + } } diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3OutputStream.java b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3OutputStream.java index 88488bf4c313..ab173d06992e 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3OutputStream.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3OutputStream.java @@ -31,7 +31,6 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; -import com.adobe.testing.s3mock.junit5.S3MockExtension; import java.io.File; import java.io.IOException; import java.io.UncheckedIOException; @@ -50,12 +49,13 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.jupiter.api.extension.RegisterExtension; import org.mockito.ArgumentCaptor; import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.testcontainers.containers.MinIOContainer; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; import software.amazon.awssdk.core.ResponseBytes; import software.amazon.awssdk.core.sync.RequestBody; import software.amazon.awssdk.core.sync.ResponseTransformer; @@ -72,16 +72,15 @@ import software.amazon.awssdk.services.s3.model.UploadPartRequest; import software.amazon.awssdk.utils.BinaryUtils; -@ExtendWith(S3MockExtension.class) +@Testcontainers public class TestS3OutputStream { private static final Logger LOG = LoggerFactory.getLogger(TestS3OutputStream.class); private static final String BUCKET = "test-bucket"; private static final int FIVE_MBS = 5 * 1024 * 1024; - @RegisterExtension - public static final S3MockExtension S3_MOCK = S3MockExtension.builder().silent().build(); + @Container private static final MinIOContainer MINIO = MinioUtil.createContainer(); - private final S3Client s3 = S3_MOCK.createS3ClientV2(); + private final S3Client s3 = MinioUtil.createS3Client(MINIO); private final S3Client s3mock = mock(S3Client.class, delegatesTo(s3)); private final Random random = new Random(1); private final Path tmpDir = Files.createTempDirectory("s3fileio-test-"); diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3URI.java b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3URI.java index 383ff67d161d..d3f8ac35d487 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3URI.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3URI.java @@ -28,6 +28,7 @@ import org.junit.jupiter.api.Test; public class TestS3URI { + private static final String S3_DIRECTORY_BUCKET = "directory-bucket-usw2-az1--x-s3"; @Test public void testLocationParsing() { @@ -96,4 +97,46 @@ public void testS3URIWithBucketToAccessPointMapping() { assertThat(uri1.key()).isEqualTo("path/to/file"); assertThat(uri1.toString()).isEqualTo(p1); } + + @Test + public void testS3URIUseS3DirectoryBucket() { + assertThat( + new S3URI(String.format("s3://%s/path/to/file", S3_DIRECTORY_BUCKET)) + .useS3DirectoryBucket()) + .isTrue(); + assertThat(new S3URI("s3://bucket/path/to/file").useS3DirectoryBucket()).isFalse(); + assertThat( + new S3URI("s3://bucket/path/to/file", ImmutableMap.of("bucket", S3_DIRECTORY_BUCKET)) + .useS3DirectoryBucket()) + .isTrue(); + assertThat( + new S3URI("s3://bucket/path/to/file", ImmutableMap.of("bucket", "bucket2")) + .useS3DirectoryBucket()) + .isFalse(); + } + + @Test + public void testS3URIToDirectoryPath() { + assertThat(new S3URI("s3://bucket/path/to/file").toDirectoryPath().location()) + .isEqualTo("s3://bucket/path/to/file/"); + assertThat(new S3URI("s3://bucket/path/to/file/").toDirectoryPath().location()) + .isEqualTo("s3://bucket/path/to/file/"); + assertThat(new S3URI("s3a://bucket/path/to/file").toDirectoryPath().location()) + .isEqualTo("s3a://bucket/path/to/file/"); + assertThat( + new S3URI(String.format("s3://%s/path/to/file", S3_DIRECTORY_BUCKET)) + .toDirectoryPath() + .location()) + .isEqualTo(String.format("s3://%s/path/to/file/", S3_DIRECTORY_BUCKET)); + assertThat( + new S3URI("s3://bucket/path/to/file", ImmutableMap.of("bucket", S3_DIRECTORY_BUCKET)) + .toDirectoryPath() + .location()) + .isEqualTo(String.format("s3://%s/path/to/file/", S3_DIRECTORY_BUCKET)); + assertThat( + new S3URI("s3://bucket/path/to/file", ImmutableMap.of("bucket", "bucket2")) + .toDirectoryPath() + .location()) + .isEqualTo("s3://bucket2/path/to/file/"); + } } diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/S3SignerServlet.java b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/S3SignerServlet.java index ce7527af765c..038d76b03e4b 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/S3SignerServlet.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/S3SignerServlet.java @@ -24,6 +24,9 @@ import static org.assertj.core.api.AssertionsForClassTypes.assertThat; import com.fasterxml.jackson.databind.ObjectMapper; +import jakarta.servlet.http.HttpServlet; +import jakarta.servlet.http.HttpServletRequest; +import jakarta.servlet.http.HttpServletResponse; import java.io.InputStreamReader; import java.io.Reader; import java.time.Clock; @@ -37,9 +40,6 @@ import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.Stream; -import javax.servlet.http.HttpServlet; -import javax.servlet.http.HttpServletRequest; -import javax.servlet.http.HttpServletResponse; import org.apache.hc.core5.http.ContentType; import org.apache.hc.core5.http.HttpHeaders; import org.apache.iceberg.exceptions.RESTException; diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3RestSigner.java b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3RestSigner.java index f2a70aee29e5..313214c4e98f 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3RestSigner.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3RestSigner.java @@ -20,6 +20,7 @@ import static org.assertj.core.api.Assertions.assertThat; +import java.net.URI; import java.nio.file.Path; import java.nio.file.Paths; import java.util.List; @@ -27,7 +28,7 @@ import java.util.Map; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.stream.Collectors; -import org.apache.iceberg.aws.s3.MinioContainer; +import org.apache.iceberg.aws.s3.MinioUtil; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.rest.auth.OAuth2Properties; @@ -41,6 +42,9 @@ import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.testcontainers.containers.MinIOContainer; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; @@ -67,15 +71,19 @@ import software.amazon.awssdk.services.s3.model.PutObjectRequest; import software.amazon.awssdk.services.s3.model.UploadPartRequest; +@Testcontainers public class TestS3RestSigner { private static final Region REGION = Region.US_WEST_2; private static final String BUCKET = "iceberg-s3-signer-test"; + static final AwsCredentialsProvider CREDENTIALS_PROVIDER = StaticCredentialsProvider.create( AwsBasicCredentials.create("accessKeyId", "secretAccessKey")); - private static final MinioContainer MINIO_CONTAINER = - new MinioContainer(CREDENTIALS_PROVIDER.resolveCredentials()); + + @Container + private static final MinIOContainer MINIO_CONTAINER = + MinioUtil.createContainer(CREDENTIALS_PROVIDER.resolveCredentials()); private static Server httpServer; private static ValidatingSigner validatingSigner; @@ -83,6 +91,8 @@ public class TestS3RestSigner { @BeforeAll public static void beforeClass() throws Exception { + assertThat(MINIO_CONTAINER.isRunning()).isTrue(); + if (null == httpServer) { httpServer = initHttpServer(); } @@ -141,7 +151,7 @@ public void before() throws Exception { s3ClientBuilder -> s3ClientBuilder.httpClientBuilder( software.amazon.awssdk.http.apache.ApacheHttpClient.builder())) - .endpointOverride(MINIO_CONTAINER.getURI()) + .endpointOverride(URI.create(MINIO_CONTAINER.getS3URL())) .forcePathStyle(true) // OSX won't resolve subdomains .overrideConfiguration( c -> c.putAdvancedOption(SdkAdvancedClientOption.SIGNER, validatingSigner)) diff --git a/azure-bundle/build.gradle b/azure-bundle/build.gradle index 9b1cd5b42fb8..46d1b3daea5b 100644 --- a/azure-bundle/build.gradle +++ b/azure-bundle/build.gradle @@ -19,7 +19,7 @@ project(":iceberg-azure-bundle") { - apply plugin: 'io.github.goooler.shadow' + apply plugin: 'com.gradleup.shadow' tasks.jar.dependsOn tasks.shadowJar diff --git a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/AzuriteContainer.java b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/AzuriteContainer.java index 887758af4b28..7653fcce73ca 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/AzuriteContainer.java +++ b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/AzuriteContainer.java @@ -33,7 +33,7 @@ public class AzuriteContainer extends GenericContainer { private static final int DEFAULT_PORT = 10000; // default blob service port private static final String DEFAULT_IMAGE = "mcr.microsoft.com/azure-storage/azurite"; - private static final String DEFAULT_TAG = "3.30.0"; + private static final String DEFAULT_TAG = "3.33.0"; private static final String LOG_WAIT_REGEX = "Azurite Blob service is successfully listening at .*"; diff --git a/baseline.gradle b/baseline.gradle index 486625636b59..07889d4e348c 100644 --- a/baseline.gradle +++ b/baseline.gradle @@ -50,16 +50,7 @@ subprojects { t.setDuplicatesStrategy(DuplicatesStrategy.WARN); }); apply plugin: 'com.palantir.baseline-exact-dependencies' - // We need to update Google Java Format to 1.17.0+ to run spotless on JDK 8, but that requires dropping support for JDK 8. - if (JavaVersion.current() == JavaVersion.VERSION_21) { - task spotlessApply { - doLast { - throw new GradleException("Spotless plugin is currently disabled when running on JDK 21 (until we drop JDK 8). To run spotlessApply please use a different JDK version.") - } - } - } else { - apply plugin: 'com.diffplug.spotless' - } + apply plugin: 'com.diffplug.spotless' pluginManager.withPlugin('com.palantir.baseline-checkstyle') { checkstyle { @@ -73,9 +64,10 @@ subprojects { pluginManager.withPlugin('com.diffplug.spotless') { spotless { java { - target 'src/main/java/**/*.java', 'src/test/java/**/*.java', 'src/jmh/java/**/*.java', 'src/integration/java/**/*.java' - // we use an older version of google-java-format that is compatible with JDK 8 - googleJavaFormat("1.7") + target 'src/main/java/**/*.java', 'src/test/java/**/*.java', 'src/testFixtures/java/**/*.java', 'src/jmh/java/**/*.java', 'src/integration/java/**/*.java' + // 1.23.0 has an issue in formatting comments https://github.com/google/google-java-format/issues/1155 + // so we stick to 1.22.0 to produce consistent result for JDK 11/17/21 + googleJavaFormat("1.22.0") removeUnusedImports() licenseHeaderFile "$rootDir/.baseline/copyright/copyright-header-java.txt" } @@ -88,18 +80,36 @@ subprojects { // error-prone is slow, don't run on tests/generated-src/generated '-XepExcludedPaths:.*/(test|generated-src|generated)/.*', '-Xep:AnnotateFormatMethod:ERROR', + '-Xep:BadComparable:ERROR', + '-Xep:BadInstanceof:ERROR', + '-Xep:CatchFail:ERROR', + '-Xep:ClassCanBeStatic:ERROR', + '-Xep:ClassNewInstance:ERROR', '-Xep:CollectionUndefinedEquality:ERROR', // specific to Palantir - Uses name `log` but we use name `LOG` '-Xep:ConsistentLoggerName:OFF', '-Xep:DangerousJavaDeserialization:ERROR', '-Xep:DangerousThreadPoolExecutorUsage:OFF', + '-Xep:DefaultCharset:ERROR', + '-Xep:DefaultLocale:ERROR', // subclasses are not equal '-Xep:EqualsGetClass:OFF', + '-Xep:EqualsUnsafeCast:ERROR', + '-Xep:EqualsUsingHashCode:ERROR', + '-Xep:ExtendsObject:ERROR', + '-Xep:FallThrough:ERROR', // specific to Palantir '-Xep:FinalClass:OFF', + '-Xep:Finalize:ERROR', '-Xep:FormatStringAnnotation:ERROR', + '-Xep:GetClassOnEnum:ERROR', + '-Xep:HidingField:ERROR', + '-Xep:ImmutableSetForContains:ERROR', '-Xep:ImmutablesReferenceEquality:ERROR', + '-Xep:InconsistentCapitalization:ERROR', + '-Xep:InconsistentHashCode:ERROR', '-Xep:IntLongMath:ERROR', + '-Xep:JdkObsolete:ERROR', // prefer method references over lambdas '-Xep:LambdaMethodReference:ERROR', // enforce logging conventions @@ -109,8 +119,18 @@ subprojects { // Enforce missing override '-Xep:MissingOverride:ERROR', '-Xep:MissingSummary:ERROR', + '-Xep:ModifiedButNotUsed:ERROR', + '-Xep:MutablePublicArray:ERROR', + '-Xep:NarrowCalculation:ERROR', + '-Xep:NarrowingCompoundAssignment:ERROR', + '-Xep:NullOptional:ERROR', + '-Xep:NullableOptional:ERROR', + '-Xep:NullablePrimitive:ERROR', + '-Xep:ObjectEqualsForPrimitives:ERROR', // Enforce hashCode over hash '-Xep:ObjectsHashCodeUnnecessaryVarargs:ERROR', + '-Xep:OrphanedFormatString:ERROR', + '-Xep:Overrides:ERROR', // Triggers false-positives whenever relocated @VisibleForTesting is used '-Xep:PreferCommonAnnotations:OFF', // specific to Palantir @@ -130,8 +150,14 @@ subprojects { '-Xep:StringSplitter:ERROR', '-Xep:TypeParameterShadowing:OFF', '-Xep:TypeParameterUnusedInFormals:OFF', + '-Xep:UnicodeEscape:ERROR', // Palantir's UnnecessarilyQualified may throw during analysis '-Xep:UnnecessarilyQualified:OFF', + '-Xep:UnnecessaryLongToIntConversion:ERROR', + '-Xep:UnnecessaryMethodReference:ERROR', + '-Xep:UnusedMethod:ERROR', + '-Xep:UnusedVariable:ERROR', + '-Xep:UseEnumSwitch:ERROR', ) } } diff --git a/build.gradle b/build.gradle index 7a11943cf8be..abab68ca4b8b 100644 --- a/build.gradle +++ b/build.gradle @@ -26,9 +26,9 @@ buildscript { gradlePluginPortal() } dependencies { - classpath 'io.github.goooler.shadow:shadow-gradle-plugin:8.1.8' - classpath 'com.palantir.baseline:gradle-baseline-java:5.61.0' - classpath 'com.diffplug.spotless:spotless-plugin-gradle:6.13.0' + classpath 'com.gradleup.shadow:shadow-gradle-plugin:8.3.3' + classpath 'com.palantir.baseline:gradle-baseline-java:5.72.0' + classpath 'com.diffplug.spotless:spotless-plugin-gradle:6.25.0' classpath 'gradle.plugin.org.inferred:gradle-processors:3.7.0' classpath 'me.champeau.jmh:jmh-gradle-plugin:0.7.2' classpath 'gradle.plugin.io.morethan.jmhreport:gradle-jmh-report:0.9.6' @@ -246,7 +246,7 @@ subprojects { } project(':iceberg-bundled-guava') { - apply plugin: 'io.github.goooler.shadow' + apply plugin: 'com.gradleup.shadow' tasks.jar.dependsOn tasks.shadowJar @@ -347,6 +347,7 @@ project(':iceberg-core') { implementation libs.jackson.core implementation libs.jackson.databind implementation libs.caffeine + implementation libs.failsafe implementation libs.roaringbitmap compileOnly(libs.hadoop2.client) { exclude group: 'org.apache.avro', module: 'avro' @@ -437,16 +438,6 @@ project(':iceberg-aliyun') { testImplementation platform(libs.jackson.bom) testImplementation "com.fasterxml.jackson.dataformat:jackson-dataformat-xml" testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') - testImplementation libs.spring.web - testImplementation(libs.spring.boot.starter.jetty) { - exclude module: 'logback-classic' - exclude group: 'org.eclipse.jetty.websocket', module: 'javax-websocket-server-impl' - exclude group: 'org.eclipse.jetty.websocket', module: 'websocket-server' - } - testImplementation(libs.spring.boot.starter.web) { - exclude module: 'logback-classic' - exclude module: 'spring-boot-starter-logging' - } } } @@ -462,6 +453,7 @@ project(':iceberg-aws') { annotationProcessor libs.immutables.value compileOnly libs.immutables.value implementation libs.caffeine + implementation libs.failsafe implementation platform(libs.jackson.bom) implementation libs.jackson.core implementation libs.jackson.databind @@ -471,6 +463,7 @@ project(':iceberg-aws') { compileOnly("software.amazon.awssdk:url-connection-client") compileOnly("software.amazon.awssdk:apache-client") compileOnly("software.amazon.awssdk:auth") + compileOnly("software.amazon.awssdk:http-auth-aws-crt") compileOnly("software.amazon.awssdk:s3") compileOnly("software.amazon.awssdk:kms") compileOnly("software.amazon.awssdk:glue") @@ -492,20 +485,18 @@ project(':iceberg-aws') { testImplementation("software.amazon.awssdk:s3control") testImplementation("software.amazon.s3.accessgrants:aws-s3-accessgrants-java-plugin") testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') - testImplementation(libs.s3mock.junit5) { - exclude module: "spring-boot-starter-logging" - exclude module: "logback-classic" - exclude group: 'junit' - } testImplementation libs.esotericsoftware.kryo testImplementation libs.sqlite.jdbc testImplementation libs.testcontainers + testImplementation libs.testcontainers.junit.jupiter + testImplementation libs.testcontainers.minio testImplementation libs.httpcomponents.httpclient5 testImplementation libs.mockserver.netty testImplementation libs.mockserver.client.java testImplementation libs.jaxb.api testImplementation project(path: ':iceberg-core', configuration: 'testArtifacts') testImplementation libs.awaitility + testImplementation libs.jetty.servlet } sourceSets { @@ -952,6 +943,53 @@ project(':iceberg-snowflake') { } project(':iceberg-open-api') { + apply plugin: 'java-test-fixtures' + + dependencies { + testImplementation project(':iceberg-api') + testImplementation project(':iceberg-core') + testImplementation project(':iceberg-core').sourceSets.test.runtimeClasspath + testImplementation(testFixtures(project(':iceberg-open-api'))) + + testImplementation libs.junit.jupiter + testImplementation libs.junit.suite.api + testImplementation libs.junit.suite.engine + testImplementation libs.assertj.core + + testImplementation project(':iceberg-aws-bundle') + testImplementation project(':iceberg-gcp-bundle') + testImplementation project(':iceberg-azure-bundle') + + testFixturesImplementation project(':iceberg-api') + testFixturesImplementation project(':iceberg-core') + testFixturesImplementation project(path: ':iceberg-core', configuration: 'testArtifacts') + testFixturesImplementation project(':iceberg-core').sourceSets.test.runtimeClasspath + testFixturesImplementation project(':iceberg-aws') + testFixturesImplementation project(':iceberg-gcp') + testFixturesImplementation project(':iceberg-azure') + + testFixturesImplementation libs.jetty.servlet + testFixturesImplementation libs.jetty.server + testFixturesImplementation libs.sqlite.jdbc + + testFixturesCompileOnly libs.apiguardian + } + + test { + useJUnitPlatform() + + // Always rerun the compatibility tests + outputs.upToDateWhen {false} + maxParallelForks = 1 + + // Pass through any system properties that start with "rck" (REST Compatibility Kit) + // Note: only pass through specific properties so they do not affect other build/test + // configurations + systemProperties System.properties + .findAll { k, v -> k.startsWith("rck") } + .collectEntries { k, v -> { [(k):v, (k.replaceFirst("rck.", "")):v] }} // strip prefix + } + def restCatalogSpec = "$projectDir/rest-catalog-open-api.yaml" tasks.register('validateRESTCatalogSpec', org.openapitools.generator.gradle.plugin.tasks.ValidateTask) { inputSpec.set(restCatalogSpec) diff --git a/common/src/main/java/org/apache/iceberg/common/DynConstructors.java b/common/src/main/java/org/apache/iceberg/common/DynConstructors.java index 7ec8716c86a4..ea2fca931f13 100644 --- a/common/src/main/java/org/apache/iceberg/common/DynConstructors.java +++ b/common/src/main/java/org/apache/iceberg/common/DynConstructors.java @@ -76,7 +76,9 @@ public R invoke(Object target, Object... args) { return (R) newInstance(args); } - /** @deprecated since 1.7.0, visibility will be reduced in 1.8.0 */ + /** + * @deprecated since 1.7.0, visibility will be reduced in 1.8.0 + */ @Deprecated // will become package-private @Override @SuppressWarnings("unchecked") diff --git a/common/src/main/java/org/apache/iceberg/common/DynMethods.java b/common/src/main/java/org/apache/iceberg/common/DynMethods.java index 65a69bd0e12c..5972cdf8dc23 100644 --- a/common/src/main/java/org/apache/iceberg/common/DynMethods.java +++ b/common/src/main/java/org/apache/iceberg/common/DynMethods.java @@ -125,7 +125,9 @@ public String toString() { /** Singleton {@link UnboundMethod}, performs no operation and returns null. */ private static final UnboundMethod NOOP = new UnboundMethod(null, "NOOP") { - /** @deprecated since 1.7.0, visibility will be reduced in 1.8.0 */ + /** + * @deprecated since 1.7.0, visibility will be reduced in 1.8.0 + */ @Deprecated // will become package-private @Override public R invokeChecked(Object target, Object... args) { diff --git a/core/src/jmh/java/org/apache/iceberg/AppendBenchmark.java b/core/src/jmh/java/org/apache/iceberg/AppendBenchmark.java new file mode 100644 index 000000000000..a444e7ff9c63 --- /dev/null +++ b/core/src/jmh/java/org/apache/iceberg/AppendBenchmark.java @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.util.List; +import java.util.concurrent.TimeUnit; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Timeout; +import org.openjdk.jmh.annotations.Warmup; + +/** + * A benchmark that evaluates the performance of appending files to the table. + * + *

To run this benchmark: + * ./gradlew :iceberg-core:jmh + * -PjmhIncludeRegex=AppendBenchmark + * -PjmhOutputPath=benchmark/append-benchmark.txt + * + */ +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 5) +@BenchmarkMode(Mode.SingleShotTime) +@Timeout(time = 10, timeUnit = TimeUnit.MINUTES) +public class AppendBenchmark { + + private static final String TABLE_IDENT = "tbl"; + private static final Schema SCHEMA = + new Schema( + required(1, "int_col", Types.IntegerType.get()), + required(2, "long_col", Types.LongType.get()), + required(3, "decimal_col", Types.DecimalType.of(10, 10)), + required(4, "date_col", Types.DateType.get()), + required(5, "timestamp_col", Types.TimestampType.withoutZone()), + required(6, "timestamp_tz_col", Types.TimestampType.withZone()), + required(7, "str_col1", Types.StringType.get()), + required(8, "str_col2", Types.StringType.get()), + required(9, "str_col3", Types.StringType.get()), + required(10, "str_col4", Types.StringType.get()), + required(11, "str_col5", Types.StringType.get()), + required(12, "str_col6", Types.StringType.get()), + required(13, "str_col7", Types.StringType.get())); + private static final PartitionSpec SPEC = PartitionSpec.unpartitioned(); + private static final HadoopTables TABLES = new HadoopTables(); + + private Table table; + private List dataFiles; + + @Param({"50000", "100000", "500000", "1000000", "2500000"}) + private int numFiles; + + @Param({"true", "false"}) + private boolean fast; + + @Setup + public void setupBenchmark() { + initTable(); + initDataFiles(); + } + + @TearDown + public void tearDownBenchmark() { + dropTable(); + } + + @Benchmark + @Threads(1) + public void appendFiles() { + AppendFiles append = fast ? table.newFastAppend() : table.newAppend(); + + for (DataFile dataFile : dataFiles) { + append.appendFile(dataFile); + } + + append.commit(); + } + + private void initTable() { + this.table = TABLES.create(SCHEMA, SPEC, TABLE_IDENT); + } + + private void dropTable() { + TABLES.dropTable(TABLE_IDENT); + } + + private void initDataFiles() { + List generatedDataFiles = Lists.newArrayListWithExpectedSize(numFiles); + + for (int ordinal = 0; ordinal < numFiles; ordinal++) { + DataFile dataFile = FileGenerationUtil.generateDataFile(table, null); + generatedDataFiles.add(dataFile); + } + + this.dataFiles = generatedDataFiles; + } +} diff --git a/core/src/jmh/java/org/apache/iceberg/ManifestReadBenchmark.java b/core/src/jmh/java/org/apache/iceberg/ManifestReadBenchmark.java index 6677e5d8b651..7b4e2b9ec523 100644 --- a/core/src/jmh/java/org/apache/iceberg/ManifestReadBenchmark.java +++ b/core/src/jmh/java/org/apache/iceberg/ManifestReadBenchmark.java @@ -69,10 +69,9 @@ public void before() { manifestListFile = String.format("%s/%s.avro", baseDir, UUID.randomUUID()); Random random = new Random(System.currentTimeMillis()); - ManifestListWriter listWriter = - ManifestLists.write(1, org.apache.iceberg.Files.localOutput(manifestListFile), 0, 1L, 0); - try { + try (ManifestListWriter listWriter = + ManifestLists.write(1, org.apache.iceberg.Files.localOutput(manifestListFile), 0, 1L, 0)) { for (int i = 0; i < NUM_FILES; i++) { OutputFile manifestFile = org.apache.iceberg.Files.localOutput( @@ -98,8 +97,6 @@ public void before() { listWriter.add(writer.toManifestFile()); } - - listWriter.close(); } catch (IOException e) { throw new UncheckedIOException(e); } @@ -125,12 +122,11 @@ public void readManifestFile() throws IOException { TestTables.LocalFileIO fileIO = new TestTables.LocalFileIO(); Map specs = ImmutableMap.of(PartitionSpec.unpartitioned().specId(), PartitionSpec.unpartitioned()); - long recordCount = 0L; for (ManifestFile manifestFile : manifests) { ManifestReader reader = ManifestFiles.read(manifestFile, fileIO, specs); try (CloseableIterator it = reader.iterator()) { while (it.hasNext()) { - recordCount += it.next().recordCount(); + it.next().recordCount(); } } } diff --git a/core/src/jmh/java/org/apache/iceberg/ManifestWriteBenchmark.java b/core/src/jmh/java/org/apache/iceberg/ManifestWriteBenchmark.java new file mode 100644 index 000000000000..c175248241e9 --- /dev/null +++ b/core/src/jmh/java/org/apache/iceberg/ManifestWriteBenchmark.java @@ -0,0 +1,170 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.Random; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import org.apache.commons.io.FileUtils; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.io.Files; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Timeout; + +/** + * A benchmark that evaluates the performance of writing manifest files + * + *

To run this benchmark: + * ./gradlew :iceberg-core:jmh -PjmhIncludeRegex=ManifestWriteBenchmark + * + */ +@Fork(1) +@State(Scope.Benchmark) +@Measurement(iterations = 5) +@BenchmarkMode(Mode.SingleShotTime) +@Timeout(time = 5, timeUnit = TimeUnit.MINUTES) +public class ManifestWriteBenchmark { + + private static final int NUM_FILES = 10; + private static final int NUM_ROWS = 100000; + private static final int NUM_COLS = 100; + + private String baseDir; + private String manifestListFile; + + private Metrics metrics; + + @Setup + public void before() { + Random random = new Random(System.currentTimeMillis()); + // Pre-create the metrics to avoid doing this in the benchmark itself + metrics = randomMetrics(random); + } + + @TearDown + public void after() { + if (baseDir != null) { + FileUtils.deleteQuietly(new File(baseDir)); + baseDir = null; + } + + manifestListFile = null; + } + + @State(Scope.Benchmark) + public static class BenchmarkState { + @Param({"1", "2"}) + private int formatVersion; + + public int getFormatVersion() { + return formatVersion; + } + } + + @Benchmark + @Threads(1) + public void writeManifestFile(BenchmarkState state) throws IOException { + this.baseDir = Files.createTempDir().getAbsolutePath(); + this.manifestListFile = String.format("%s/%s.avro", baseDir, UUID.randomUUID()); + + try (ManifestListWriter listWriter = + ManifestLists.write( + state.getFormatVersion(), + org.apache.iceberg.Files.localOutput(manifestListFile), + 0, + 1L, + 0)) { + for (int i = 0; i < NUM_FILES; i++) { + OutputFile manifestFile = + org.apache.iceberg.Files.localOutput( + String.format("%s/%s.avro", baseDir, UUID.randomUUID())); + + ManifestWriter writer = + ManifestFiles.write( + state.formatVersion, PartitionSpec.unpartitioned(), manifestFile, 1L); + try (ManifestWriter finalWriter = writer) { + for (int j = 0; j < NUM_ROWS; j++) { + DataFile dataFile = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withFormat(FileFormat.PARQUET) + .withPath(String.format("/path/to/data-%s-%s.parquet", i, j)) + .withFileSizeInBytes(j) + .withRecordCount(j) + .withMetrics(metrics) + .build(); + finalWriter.add(dataFile); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + + listWriter.add(writer.toManifestFile()); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + private Metrics randomMetrics(Random random) { + long rowCount = 100000L + random.nextInt(1000); + Map columnSizes = Maps.newHashMap(); + Map valueCounts = Maps.newHashMap(); + Map nullValueCounts = Maps.newHashMap(); + Map nanValueCounts = Maps.newHashMap(); + Map lowerBounds = Maps.newHashMap(); + Map upperBounds = Maps.newHashMap(); + for (int i = 0; i < NUM_COLS; i++) { + columnSizes.put(i, 1000000L + random.nextInt(100000)); + valueCounts.put(i, 100000L + random.nextInt(100)); + nullValueCounts.put(i, (long) random.nextInt(5)); + nanValueCounts.put(i, (long) random.nextInt(5)); + byte[] lower = new byte[8]; + random.nextBytes(lower); + lowerBounds.put(i, ByteBuffer.wrap(lower)); + byte[] upper = new byte[8]; + random.nextBytes(upper); + upperBounds.put(i, ByteBuffer.wrap(upper)); + } + + return new Metrics( + rowCount, + columnSizes, + valueCounts, + nullValueCounts, + nanValueCounts, + lowerBounds, + upperBounds); + } +} diff --git a/core/src/jmh/java/org/apache/iceberg/PartitionStatsUtilBenchmark.java b/core/src/jmh/java/org/apache/iceberg/PartitionStatsUtilBenchmark.java new file mode 100644 index 000000000000..539494e34735 --- /dev/null +++ b/core/src/jmh/java/org/apache/iceberg/PartitionStatsUtilBenchmark.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Collection; +import java.util.concurrent.TimeUnit; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.types.Types; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Timeout; +import org.openjdk.jmh.annotations.Warmup; + +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 2) +@Measurement(iterations = 5) +@Timeout(time = 1000, timeUnit = TimeUnit.HOURS) +@BenchmarkMode(Mode.SingleShotTime) +public class PartitionStatsUtilBenchmark { + + private static final Schema SCHEMA = + new Schema( + required(1, "c1", Types.IntegerType.get()), + optional(2, "c2", Types.StringType.get()), + optional(3, "c3", Types.StringType.get())); + + private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build(); + + // Create 10k manifests + private static final int MANIFEST_COUNTER = 10000; + + // each manifest with 100 partition values + private static final int PARTITION_PER_MANIFEST = 100; + + // 20 data files per partition, which results in 2k data files per manifest + private static final int DATA_FILES_PER_PARTITION_COUNT = 20; + + private static final HadoopTables TABLES = new HadoopTables(); + + private static final String TABLE_IDENT = "tbl"; + + private Table table; + + @Setup + public void setupBenchmark() { + this.table = TABLES.create(SCHEMA, SPEC, TABLE_IDENT); + + for (int manifestCount = 0; manifestCount < MANIFEST_COUNTER; manifestCount++) { + AppendFiles appendFiles = table.newFastAppend(); + + for (int partition = 0; partition < PARTITION_PER_MANIFEST; partition++) { + StructLike partitionData = TestHelpers.Row.of(partition); + for (int fileOrdinal = 0; fileOrdinal < DATA_FILES_PER_PARTITION_COUNT; fileOrdinal++) { + appendFiles.appendFile(FileGenerationUtil.generateDataFile(table, partitionData)); + } + } + + appendFiles.commit(); + } + } + + @TearDown + public void tearDownBenchmark() { + TABLES.dropTable(TABLE_IDENT); + } + + @Benchmark + @Threads(1) + public void benchmarkPartitionStats() { + Collection partitionStats = + PartitionStatsUtil.computeStats(table, table.currentSnapshot()); + assertThat(partitionStats).hasSize(PARTITION_PER_MANIFEST); + + PartitionStatsUtil.sortStats(partitionStats, Partitioning.partitionType(table)); + } +} diff --git a/core/src/jmh/java/org/apache/iceberg/ReplaceDeleteFilesBenchmark.java b/core/src/jmh/java/org/apache/iceberg/ReplaceDeleteFilesBenchmark.java new file mode 100644 index 000000000000..a899b870a90c --- /dev/null +++ b/core/src/jmh/java/org/apache/iceberg/ReplaceDeleteFilesBenchmark.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.util.List; +import java.util.concurrent.TimeUnit; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Timeout; +import org.openjdk.jmh.annotations.Warmup; + +/** + * A benchmark that evaluates the performance of replacing delete files in the table. + * + *

To run this benchmark: + * ./gradlew :iceberg-core:jmh + * -PjmhIncludeRegex=ReplaceDeleteFilesBenchmark + * -PjmhOutputPath=benchmark/replace-delete-files-benchmark.txt + * + */ +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 5) +@BenchmarkMode(Mode.SingleShotTime) +@Timeout(time = 10, timeUnit = TimeUnit.MINUTES) +public class ReplaceDeleteFilesBenchmark { + + private static final String TABLE_IDENT = "tbl"; + private static final Schema SCHEMA = + new Schema( + required(1, "int_col", Types.IntegerType.get()), + required(2, "long_col", Types.LongType.get()), + required(3, "decimal_col", Types.DecimalType.of(10, 10)), + required(4, "date_col", Types.DateType.get()), + required(5, "timestamp_col", Types.TimestampType.withoutZone()), + required(6, "timestamp_tz_col", Types.TimestampType.withZone()), + required(7, "str_col", Types.StringType.get())); + private static final PartitionSpec SPEC = PartitionSpec.unpartitioned(); + private static final HadoopTables TABLES = new HadoopTables(); + + private Table table; + private List deleteFiles; + private List pendingDeleteFiles; + + @Param({"50000", "100000", "500000", "1000000", "2500000"}) + private int numFiles; + + @Setup + public void setupBenchmark() { + initTable(); + initFiles(); + } + + @TearDown + public void tearDownBenchmark() { + dropTable(); + } + + @Benchmark + @Threads(1) + public void replaceDeleteFiles() { + RowDelta rowDelta = table.newRowDelta(); + deleteFiles.forEach(rowDelta::removeDeletes); + pendingDeleteFiles.forEach(rowDelta::addDeletes); + rowDelta.commit(); + } + + private void initTable() { + this.table = TABLES.create(SCHEMA, SPEC, TABLE_IDENT); + } + + private void dropTable() { + TABLES.dropTable(TABLE_IDENT); + } + + private void initFiles() { + List generatedDeleteFiles = Lists.newArrayListWithExpectedSize(numFiles); + List generatedPendingDeleteFiles = Lists.newArrayListWithExpectedSize(numFiles); + + RowDelta rowDelta = table.newRowDelta(); + + for (int ordinal = 0; ordinal < numFiles; ordinal++) { + DataFile dataFile = FileGenerationUtil.generateDataFile(table, null); + rowDelta.addRows(dataFile); + + DeleteFile deleteFile = FileGenerationUtil.generatePositionDeleteFile(table, dataFile); + rowDelta.addDeletes(deleteFile); + generatedDeleteFiles.add(deleteFile); + + DeleteFile pendingDeleteFile = FileGenerationUtil.generatePositionDeleteFile(table, dataFile); + generatedPendingDeleteFiles.add(pendingDeleteFile); + } + + rowDelta.commit(); + + this.deleteFiles = generatedDeleteFiles; + this.pendingDeleteFiles = generatedPendingDeleteFiles; + } +} diff --git a/core/src/jmh/java/org/apache/iceberg/deletes/RoaringPositionBitmapBenchmark.java b/core/src/jmh/java/org/apache/iceberg/deletes/RoaringPositionBitmapBenchmark.java new file mode 100644 index 000000000000..1cbc39583fbc --- /dev/null +++ b/core/src/jmh/java/org/apache/iceberg/deletes/RoaringPositionBitmapBenchmark.java @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.deletes; + +import java.util.Random; +import java.util.concurrent.TimeUnit; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Timeout; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; +import org.roaringbitmap.longlong.Roaring64Bitmap; + +/** + * A benchmark that evaluates the performance of {@link RoaringPositionBitmap}. + * + *

To run this benchmark: + * ./gradlew :iceberg-core:jmh + * -PjmhIncludeRegex=RoaringPositionBitmapBenchmark + * -PjmhOutputPath=benchmark/roaring-position-bitmap-benchmark.txt + * + */ +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 5) +@BenchmarkMode(Mode.SingleShotTime) +@Timeout(time = 5, timeUnit = TimeUnit.MINUTES) +public class RoaringPositionBitmapBenchmark { + + private static final Random RANDOM = new Random(); + private static final int TOTAL_POSITIONS = 5_000_000; + private static final long STEP = 5L; + + private long[] orderedPositions; + private long[] shuffledPositions; + + @Setup + public void setupBenchmark() { + this.orderedPositions = generateOrderedPositions(); + this.shuffledPositions = generateShuffledPositions(); + } + + @Benchmark + @Threads(1) + public void addOrderedPositionsIcebergBitmap(Blackhole blackhole) { + RoaringPositionBitmap bitmap = new RoaringPositionBitmap(); + for (long position : orderedPositions) { + bitmap.set(position); + } + blackhole.consume(bitmap); + } + + @Benchmark + @Threads(1) + public void addOrderedPositionsLibraryBitmap(Blackhole blackhole) { + Roaring64Bitmap bitmap = new Roaring64Bitmap(); + for (long position : orderedPositions) { + bitmap.add(position); + } + blackhole.consume(bitmap); + } + + @Benchmark + @Threads(1) + public void addShuffledPositionsIcebergBitmap(Blackhole blackhole) { + RoaringPositionBitmap bitmap = new RoaringPositionBitmap(); + for (long position : shuffledPositions) { + bitmap.set(position); + } + blackhole.consume(bitmap); + } + + @Benchmark + @Threads(1) + public void addShuffledPositionsLibraryBitmap(Blackhole blackhole) { + Roaring64Bitmap bitmap = new Roaring64Bitmap(); + for (long position : shuffledPositions) { + bitmap.add(position); + } + blackhole.consume(bitmap); + } + + @Benchmark + @Threads(1) + public void addAndCheckPositionsIcebergBitmap(Blackhole blackhole) { + RoaringPositionBitmap bitmap = new RoaringPositionBitmap(); + + for (long position : shuffledPositions) { + bitmap.set(position); + } + + for (long position = 0; position <= TOTAL_POSITIONS * STEP; position++) { + bitmap.contains(position); + } + + blackhole.consume(bitmap); + } + + @Benchmark + @Threads(1) + public void addAndCheckPositionsLibraryBitmap(Blackhole blackhole) { + Roaring64Bitmap bitmap = new Roaring64Bitmap(); + + for (long position : shuffledPositions) { + bitmap.add(position); + } + + for (long position = 0; position <= TOTAL_POSITIONS * STEP; position++) { + bitmap.contains(position); + } + + blackhole.consume(bitmap); + } + + private static long[] generateOrderedPositions() { + long[] positions = new long[TOTAL_POSITIONS]; + for (int index = 0; index < TOTAL_POSITIONS; index++) { + positions[index] = index * STEP; + } + return positions; + } + + private static long[] generateShuffledPositions() { + long[] positions = generateOrderedPositions(); + shuffle(positions); + return positions; + } + + private static void shuffle(long[] array) { + for (int index = array.length - 1; index > 0; index--) { + // swap with an element at a random index between 0 and index + int thatIndex = RANDOM.nextInt(index + 1); + long temp = array[index]; + array[index] = array[thatIndex]; + array[thatIndex] = temp; + } + } +} diff --git a/core/src/jmh/java/org/apache/iceberg/metrics/CountersBenchmark.java b/core/src/jmh/java/org/apache/iceberg/metrics/CountersBenchmark.java index 1350950d4874..628028d9f8e6 100644 --- a/core/src/jmh/java/org/apache/iceberg/metrics/CountersBenchmark.java +++ b/core/src/jmh/java/org/apache/iceberg/metrics/CountersBenchmark.java @@ -50,7 +50,7 @@ public class CountersBenchmark { public void defaultCounterMultipleThreads(Blackhole blackhole) { Counter counter = new DefaultCounter(Unit.BYTES); - ExecutorService workerPool = ThreadPools.newWorkerPool("bench-pool", WORKER_POOL_SIZE); + ExecutorService workerPool = ThreadPools.newFixedThreadPool("bench-pool", WORKER_POOL_SIZE); try { Tasks.range(WORKER_POOL_SIZE) diff --git a/core/src/main/java/org/apache/iceberg/BaseDistributedDataScan.java b/core/src/main/java/org/apache/iceberg/BaseDistributedDataScan.java index 263830a53b3c..89c7f0b6067e 100644 --- a/core/src/main/java/org/apache/iceberg/BaseDistributedDataScan.java +++ b/core/src/main/java/org/apache/iceberg/BaseDistributedDataScan.java @@ -401,8 +401,14 @@ private boolean mayHaveEqualityDeletes(Snapshot snapshot) { return count == null || !count.equals("0"); } - // a monitor pool that enables planing data and deletes concurrently if remote planning is used + /** + * Creates a monitor pool that enables planing data and deletes concurrently if remote planning is + * used + * + *

Important: Callers are responsible for shutting down the returned executor service + * when it is no longer needed + */ private ExecutorService newMonitorPool() { - return ThreadPools.newWorkerPool("iceberg-planning-monitor-service", MONITOR_POOL_SIZE); + return ThreadPools.newFixedThreadPool("iceberg-planning-monitor-service", MONITOR_POOL_SIZE); } } diff --git a/core/src/main/java/org/apache/iceberg/BaseFile.java b/core/src/main/java/org/apache/iceberg/BaseFile.java index d4583b31c7a1..8f84eb5737b9 100644 --- a/core/src/main/java/org/apache/iceberg/BaseFile.java +++ b/core/src/main/java/org/apache/iceberg/BaseFile.java @@ -30,8 +30,8 @@ import org.apache.avro.generic.IndexedRecord; import org.apache.avro.specific.SpecificData; import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.avro.SupportsIndexProjection; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.ArrayUtil; @@ -39,7 +39,7 @@ import org.apache.iceberg.util.SerializableMap; /** Base class for both {@link DataFile} and {@link DeleteFile}. */ -abstract class BaseFile +abstract class BaseFile extends SupportsIndexProjection implements ContentFile, IndexedRecord, StructLike, @@ -55,10 +55,10 @@ public PartitionData copy() { } }; - private int[] fromProjectionPos; private Types.StructType partitionType; private Long fileOrdinal = null; + private String manifestLocation = null; private int partitionSpecId = -1; private FileContent content = FileContent.DATA; private String filePath = null; @@ -84,40 +84,51 @@ public PartitionData copy() { // cached schema private transient Schema avroSchema = null; + // struct type that corresponds to the positions used for internalGet and internalSet + private static final Types.StructType BASE_TYPE = + Types.StructType.of( + DataFile.CONTENT, + DataFile.FILE_PATH, + DataFile.FILE_FORMAT, + DataFile.SPEC_ID, + Types.NestedField.required( + DataFile.PARTITION_ID, + DataFile.PARTITION_NAME, + EMPTY_STRUCT_TYPE, + DataFile.PARTITION_DOC), + DataFile.RECORD_COUNT, + DataFile.FILE_SIZE, + DataFile.COLUMN_SIZES, + DataFile.VALUE_COUNTS, + DataFile.NULL_VALUE_COUNTS, + DataFile.NAN_VALUE_COUNTS, + DataFile.LOWER_BOUNDS, + DataFile.UPPER_BOUNDS, + DataFile.KEY_METADATA, + DataFile.SPLIT_OFFSETS, + DataFile.EQUALITY_IDS, + DataFile.SORT_ORDER_ID, + MetadataColumns.ROW_POSITION); + /** Used by Avro reflection to instantiate this class when reading manifest files. */ BaseFile(Schema avroSchema) { + this(AvroSchemaUtil.convert(avroSchema).asStructType()); this.avroSchema = avroSchema; + } - Types.StructType schema = AvroSchemaUtil.convert(avroSchema).asNestedType().asStructType(); + /** Used by internal readers to instantiate this class with a projection schema. */ + BaseFile(Types.StructType projection) { + super(BASE_TYPE, projection); + this.avroSchema = AvroSchemaUtil.convert(projection, "data_file"); // partition type may be null if the field was not projected - Type partType = schema.fieldType("partition"); + Type partType = projection.fieldType("partition"); if (partType != null) { this.partitionType = partType.asNestedType().asStructType(); } else { this.partitionType = EMPTY_STRUCT_TYPE; } - List fields = schema.fields(); - List allFields = Lists.newArrayList(); - allFields.addAll(DataFile.getType(partitionType).fields()); - allFields.add(MetadataColumns.ROW_POSITION); - - this.fromProjectionPos = new int[fields.size()]; - for (int i = 0; i < fromProjectionPos.length; i += 1) { - boolean found = false; - for (int j = 0; j < allFields.size(); j += 1) { - if (fields.get(i).fieldId() == allFields.get(j).fieldId()) { - found = true; - fromProjectionPos[i] = j; - } - } - - if (!found) { - throw new IllegalArgumentException("Cannot find projected field: " + fields.get(i)); - } - } - this.partitionData = new PartitionData(partitionType); } @@ -139,6 +150,7 @@ public PartitionData copy() { int[] equalityFieldIds, Integer sortOrderId, ByteBuffer keyMetadata) { + super(BASE_TYPE.fields().size()); this.partitionSpecId = specId; this.content = content; this.filePath = filePath; @@ -177,7 +189,9 @@ public PartitionData copy() { * column stat is kept. */ BaseFile(BaseFile toCopy, boolean copyStats, Set requestedColumnIds) { + super(toCopy); this.fileOrdinal = toCopy.fileOrdinal; + this.manifestLocation = toCopy.manifestLocation; this.partitionSpecId = toCopy.partitionSpecId; this.content = toCopy.content; this.filePath = toCopy.filePath; @@ -201,7 +215,6 @@ public PartitionData copy() { this.lowerBounds = null; this.upperBounds = null; } - this.fromProjectionPos = toCopy.fromProjectionPos; this.keyMetadata = toCopy.keyMetadata == null ? null @@ -220,7 +233,9 @@ public PartitionData copy() { } /** Constructor for Java serialization. */ - BaseFile() {} + BaseFile() { + super(BASE_TYPE.fields().size()); + } @Override public int specId() { @@ -240,6 +255,10 @@ public void setDataSequenceNumber(Long dataSequenceNumber) { this.dataSequenceNumber = dataSequenceNumber; } + void setManifestLocation(String manifestLocation) { + this.manifestLocation = manifestLocation; + } + @Override public Long fileSequenceNumber() { return fileSequenceNumber; @@ -260,13 +279,12 @@ public Schema getSchema() { } @Override - @SuppressWarnings("unchecked") public void put(int i, Object value) { - int pos = i; - // if the schema was projected, map the incoming ordinal to the expected one - if (fromProjectionPos != null) { - pos = fromProjectionPos[i]; - } + set(i, value); + } + + @Override + protected void internalSet(int pos, T value) { switch (pos) { case 0: this.content = value != null ? FILE_CONTENT_VALUES[(Integer) value] : FileContent.DATA; @@ -329,18 +347,12 @@ public void put(int i, Object value) { } @Override - public void set(int pos, T value) { - put(pos, value); + protected T internalGet(int pos, Class javaClass) { + return javaClass.cast(getByPos(pos)); } - @Override - public Object get(int i) { - int pos = i; - // if the schema was projected, map the incoming ordinal to the expected one - if (fromProjectionPos != null) { - pos = fromProjectionPos[i]; - } - switch (pos) { + private Object getByPos(int basePos) { + switch (basePos) { case 0: return content.id(); case 1: @@ -378,13 +390,13 @@ public Object get(int i) { case 17: return fileOrdinal; default: - throw new UnsupportedOperationException("Unknown field ordinal: " + pos); + throw new UnsupportedOperationException("Unknown field ordinal: " + basePos); } } @Override - public T get(int pos, Class javaClass) { - return javaClass.cast(get(pos)); + public Object get(int pos) { + return get(pos, Object.class); } @Override @@ -397,6 +409,11 @@ public Long pos() { return fileOrdinal; } + @Override + public String manifestLocation() { + return manifestLocation; + } + @Override public FileContent content() { return content; diff --git a/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java b/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java index 5c82bc877a15..dbab9e813966 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java @@ -18,7 +18,7 @@ */ package org.apache.iceberg; -import java.util.Set; +import java.util.Locale; import java.util.UUID; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; @@ -31,11 +31,8 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.io.OutputFile; -import org.apache.iceberg.io.SupportsBulkOperations; import org.apache.iceberg.relocated.com.google.common.base.Objects; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.LocationUtil; import org.apache.iceberg.util.Tasks; import org.slf4j.Logger; @@ -126,7 +123,7 @@ public void commit(TableMetadata base, TableMetadata metadata) { long start = System.currentTimeMillis(); doCommit(base, metadata); - deleteRemovedMetadataFiles(base, metadata); + CatalogUtil.deleteRemovedMetadataFiles(io(), base, metadata); requestRefresh(); LOG.info( @@ -327,7 +324,8 @@ private String newTableMetadataFilePath(TableMetadata meta, int newVersion) { TableProperties.METADATA_COMPRESSION, TableProperties.METADATA_COMPRESSION_DEFAULT); String fileExtension = TableMetadataParser.getFileExtension(codecName); return metadataFileLocation( - meta, String.format("%05d-%s%s", newVersion, UUID.randomUUID(), fileExtension)); + meta, + String.format(Locale.ROOT, "%05d-%s%s", newVersion, UUID.randomUUID(), fileExtension)); } /** @@ -352,47 +350,4 @@ private static int parseVersion(String metadataLocation) { return -1; } } - - /** - * Deletes the oldest metadata files if {@link - * TableProperties#METADATA_DELETE_AFTER_COMMIT_ENABLED} is true. - * - * @param base table metadata on which previous versions were based - * @param metadata new table metadata with updated previous versions - */ - private void deleteRemovedMetadataFiles(TableMetadata base, TableMetadata metadata) { - if (base == null) { - return; - } - - boolean deleteAfterCommit = - metadata.propertyAsBoolean( - TableProperties.METADATA_DELETE_AFTER_COMMIT_ENABLED, - TableProperties.METADATA_DELETE_AFTER_COMMIT_ENABLED_DEFAULT); - - if (deleteAfterCommit) { - Set removedPreviousMetadataFiles = - Sets.newHashSet(base.previousFiles()); - // TableMetadata#addPreviousFile builds up the metadata log and uses - // TableProperties.METADATA_PREVIOUS_VERSIONS_MAX to determine how many files should stay in - // the log, thus we don't include metadata.previousFiles() for deletion - everything else can - // be removed - removedPreviousMetadataFiles.removeAll(metadata.previousFiles()); - if (io() instanceof SupportsBulkOperations) { - ((SupportsBulkOperations) io()) - .deleteFiles( - Iterables.transform( - removedPreviousMetadataFiles, TableMetadata.MetadataLogEntry::file)); - } else { - Tasks.foreach(removedPreviousMetadataFiles) - .noRetry() - .suppressFailureWhenFinished() - .onFailure( - (previousMetadataFile, exc) -> - LOG.warn( - "Delete failed for previous metadata file: {}", previousMetadataFile, exc)) - .run(previousMetadataFile -> io().deleteFile(previousMetadataFile.file())); - } - } - } } diff --git a/core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java b/core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java index d929bc068ec2..16fbc0dd1ebc 100644 --- a/core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java +++ b/core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg; -import java.util.Set; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Evaluator; import org.apache.iceberg.expressions.Expression; @@ -26,11 +25,11 @@ import org.apache.iceberg.expressions.Projections; import org.apache.iceberg.expressions.StrictMetricsEvaluator; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.DataFileSet; public class BaseOverwriteFiles extends MergingSnapshotProducer implements OverwriteFiles { - private final Set deletedDataFiles = Sets.newHashSet(); + private final DataFileSet deletedDataFiles = DataFileSet.create(); private boolean validateAddedFilesMatchOverwriteFilter = false; private Long startingSnapshotId = null; private Expression conflictDetectionFilter = null; diff --git a/core/src/main/java/org/apache/iceberg/BaseRewriteFiles.java b/core/src/main/java/org/apache/iceberg/BaseRewriteFiles.java index d231536d0642..b25681de4238 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRewriteFiles.java +++ b/core/src/main/java/org/apache/iceberg/BaseRewriteFiles.java @@ -21,10 +21,10 @@ import java.util.Set; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.DataFileSet; class BaseRewriteFiles extends MergingSnapshotProducer implements RewriteFiles { - private final Set replacedDataFiles = Sets.newHashSet(); + private final DataFileSet replacedDataFiles = DataFileSet.create(); private Long startingSnapshotId = null; BaseRewriteFiles(String tableName, TableOperations ops) { diff --git a/core/src/main/java/org/apache/iceberg/BaseRowDelta.java b/core/src/main/java/org/apache/iceberg/BaseRowDelta.java index 42fd17f0320b..85c2269ee526 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRowDelta.java +++ b/core/src/main/java/org/apache/iceberg/BaseRowDelta.java @@ -62,6 +62,12 @@ public RowDelta addDeletes(DeleteFile deletes) { return this; } + @Override + public RowDelta removeDeletes(DeleteFile deletes) { + delete(deletes); + return this; + } + @Override public RowDelta validateFromSnapshot(long snapshotId) { this.startingSnapshotId = snapshotId; diff --git a/core/src/main/java/org/apache/iceberg/BaseScan.java b/core/src/main/java/org/apache/iceberg/BaseScan.java index 8c309cf69e6d..804df01d31ba 100644 --- a/core/src/main/java/org/apache/iceberg/BaseScan.java +++ b/core/src/main/java/org/apache/iceberg/BaseScan.java @@ -289,4 +289,21 @@ private static Schema lazyColumnProjection(TableScanContext context, Schema sche public ThisT metricsReporter(MetricsReporter reporter) { return newRefinedScan(table, schema, context.reportWith(reporter)); } + + /** + * Retrieves a list of column names based on the type of manifest content provided. + * + * @param content the manifest content type to scan. + * @return a list of column names corresponding to the specified manifest content type. + */ + static List scanColumns(ManifestContent content) { + switch (content) { + case DATA: + return BaseScan.SCAN_COLUMNS; + case DELETES: + return BaseScan.DELETE_SCAN_COLUMNS; + default: + throw new UnsupportedOperationException("Cannot read unknown manifest type: " + content); + } + } } diff --git a/core/src/main/java/org/apache/iceberg/CatalogProperties.java b/core/src/main/java/org/apache/iceberg/CatalogProperties.java index b6fd990f0ac6..339c59b45d1b 100644 --- a/core/src/main/java/org/apache/iceberg/CatalogProperties.java +++ b/core/src/main/java/org/apache/iceberg/CatalogProperties.java @@ -124,6 +124,7 @@ private CatalogProperties() {} "client.pool.cache.eviction-interval-ms"; public static final long CLIENT_POOL_CACHE_EVICTION_INTERVAL_MS_DEFAULT = TimeUnit.MINUTES.toMillis(5); + /** * A comma separated list of elements used, in addition to the {@link #URI}, to compose the key of * the client pool cache. diff --git a/core/src/main/java/org/apache/iceberg/CatalogUtil.java b/core/src/main/java/org/apache/iceberg/CatalogUtil.java index d4fcbda0686d..609e94b7b150 100644 --- a/core/src/main/java/org/apache/iceberg/CatalogUtil.java +++ b/core/src/main/java/org/apache/iceberg/CatalogUtil.java @@ -47,6 +47,7 @@ import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.Tasks; import org.apache.iceberg.util.ThreadPools; +import org.apache.iceberg.view.ViewMetadata; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -137,6 +138,23 @@ public static void dropTableData(FileIO io, TableMetadata metadata) { deleteFile(io, metadata.metadataFileLocation(), "metadata"); } + /** + * Drops view metadata files referenced by ViewMetadata. + * + *

This should be called by dropView implementations + * + * @param io a FileIO to use for deletes + * @param metadata the last valid ViewMetadata instance for a dropped view. + */ + public static void dropViewMetadata(FileIO io, ViewMetadata metadata) { + boolean gcEnabled = + PropertyUtil.propertyAsBoolean(metadata.properties(), GC_ENABLED, GC_ENABLED_DEFAULT); + + if (gcEnabled) { + deleteFile(io, metadata.metadataFileLocation(), "metadata"); + } + } + @SuppressWarnings("DangerousStringInternUsage") private static void deleteFiles(FileIO io, Set allManifests) { // keep track of deleted files in a map that can be cleaned up when memory runs low @@ -497,4 +515,49 @@ public static String fullTableName(String catalogName, TableIdentifier identifie return sb.toString(); } + + /** + * Deletes the oldest metadata files if {@link + * TableProperties#METADATA_DELETE_AFTER_COMMIT_ENABLED} is true. + * + * @param io FileIO instance to use for deletes + * @param base table metadata on which previous versions were based + * @param metadata new table metadata with updated previous versions + */ + public static void deleteRemovedMetadataFiles( + FileIO io, TableMetadata base, TableMetadata metadata) { + if (base == null) { + return; + } + + boolean deleteAfterCommit = + metadata.propertyAsBoolean( + TableProperties.METADATA_DELETE_AFTER_COMMIT_ENABLED, + TableProperties.METADATA_DELETE_AFTER_COMMIT_ENABLED_DEFAULT); + + if (deleteAfterCommit) { + Set removedPreviousMetadataFiles = + Sets.newHashSet(base.previousFiles()); + // TableMetadata#addPreviousFile builds up the metadata log and uses + // TableProperties.METADATA_PREVIOUS_VERSIONS_MAX to determine how many files should stay in + // the log, thus we don't include metadata.previousFiles() for deletion - everything else can + // be removed + removedPreviousMetadataFiles.removeAll(metadata.previousFiles()); + if (io instanceof SupportsBulkOperations) { + ((SupportsBulkOperations) io) + .deleteFiles( + Iterables.transform( + removedPreviousMetadataFiles, TableMetadata.MetadataLogEntry::file)); + } else { + Tasks.foreach(removedPreviousMetadataFiles) + .noRetry() + .suppressFailureWhenFinished() + .onFailure( + (previousMetadataFile, exc) -> + LOG.warn( + "Delete failed for previous metadata file: {}", previousMetadataFile, exc)) + .run(previousMetadataFile -> io.deleteFile(previousMetadataFile.file())); + } + } + } } diff --git a/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java b/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java index c26716481836..8444b91eecd4 100644 --- a/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java +++ b/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java @@ -51,7 +51,6 @@ import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.ArrayUtil; -import org.apache.iceberg.util.CharSequenceMap; import org.apache.iceberg.util.ContentFileUtil; import org.apache.iceberg.util.PartitionMap; import org.apache.iceberg.util.PartitionSet; @@ -70,27 +69,37 @@ class DeleteFileIndex { private final EqualityDeletes globalDeletes; private final PartitionMap eqDeletesByPartition; private final PartitionMap posDeletesByPartition; - private final CharSequenceMap posDeletesByPath; + private final Map posDeletesByPath; + private final boolean hasEqDeletes; + private final boolean hasPosDeletes; private final boolean isEmpty; private DeleteFileIndex( EqualityDeletes globalDeletes, PartitionMap eqDeletesByPartition, PartitionMap posDeletesByPartition, - CharSequenceMap posDeletesByPath) { + Map posDeletesByPath) { this.globalDeletes = globalDeletes; this.eqDeletesByPartition = eqDeletesByPartition; this.posDeletesByPartition = posDeletesByPartition; this.posDeletesByPath = posDeletesByPath; - boolean noEqDeletes = globalDeletes == null && eqDeletesByPartition == null; - boolean noPosDeletes = posDeletesByPartition == null && posDeletesByPath == null; - this.isEmpty = noEqDeletes && noPosDeletes; + this.hasEqDeletes = globalDeletes != null || eqDeletesByPartition != null; + this.hasPosDeletes = posDeletesByPartition != null || posDeletesByPath != null; + this.isEmpty = !hasEqDeletes && !hasPosDeletes; } public boolean isEmpty() { return isEmpty; } + public boolean hasEqualityDeletes() { + return hasEqDeletes; + } + + public boolean hasPositionDeletes() { + return hasPosDeletes; + } + public Iterable referencedDeleteFiles() { Iterable deleteFiles = Collections.emptyList(); @@ -167,7 +176,7 @@ private DeleteFile[] findPathDeletes(long seq, DataFile dataFile) { return EMPTY_DELETES; } - PositionDeletes deletes = posDeletesByPath.get(dataFile.path()); + PositionDeletes deletes = posDeletesByPath.get(dataFile.location()); return deletes == null ? EMPTY_DELETES : deletes.filter(seq); } @@ -424,7 +433,7 @@ DeleteFileIndex build() { EqualityDeletes globalDeletes = new EqualityDeletes(); PartitionMap eqDeletesByPartition = PartitionMap.create(specsById); PartitionMap posDeletesByPartition = PartitionMap.create(specsById); - CharSequenceMap posDeletesByPath = CharSequenceMap.create(); + Map posDeletesByPath = Maps.newHashMap(); for (DeleteFile file : files) { switch (file.content()) { @@ -448,14 +457,14 @@ DeleteFileIndex build() { } private void add( - CharSequenceMap deletesByPath, + Map deletesByPath, PartitionMap deletesByPartition, DeleteFile file) { - CharSequence path = ContentFileUtil.referencedDataFile(file); + String path = ContentFileUtil.referencedDataFileLocation(file); PositionDeletes deletes; if (path != null) { - deletes = deletesByPath.computeIfAbsent(path, PositionDeletes::new); + deletes = deletesByPath.computeIfAbsent(path, ignored -> new PositionDeletes()); } else { int specId = file.specId(); StructLike partition = file.partition(); diff --git a/core/src/main/java/org/apache/iceberg/FastAppend.java b/core/src/main/java/org/apache/iceberg/FastAppend.java index 4976a8081c44..1e2f6fe0d90d 100644 --- a/core/src/main/java/org/apache/iceberg/FastAppend.java +++ b/core/src/main/java/org/apache/iceberg/FastAppend.java @@ -24,27 +24,20 @@ import java.util.Set; import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.events.CreateSnapshotEvent; -import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.util.CharSequenceSet; +import org.apache.iceberg.util.DataFileSet; -/** - * {@link AppendFiles Append} implementation that adds a new manifest file for the write. - * - *

This implementation will attempt to commit 5 times before throwing {@link - * CommitFailedException}. - */ +/** {@link AppendFiles Append} implementation that adds a new manifest file for the write. */ class FastAppend extends SnapshotProducer implements AppendFiles { private final String tableName; private final TableOperations ops; private final PartitionSpec spec; private final SnapshotSummary.Builder summaryBuilder = SnapshotSummary.builder(); - private final List newFiles = Lists.newArrayList(); - private final CharSequenceSet newFilePaths = CharSequenceSet.empty(); + private final DataFileSet newFiles = DataFileSet.create(); private final List appendManifests = Lists.newArrayList(); private final List rewrittenAppendManifests = Lists.newArrayList(); private List newManifests = null; @@ -86,9 +79,8 @@ protected Map summary() { @Override public FastAppend appendFile(DataFile file) { Preconditions.checkNotNull(file, "Invalid data file: null"); - if (newFilePaths.add(file.path())) { + if (newFiles.add(file)) { this.hasNewFiles = true; - newFiles.add(file); summaryBuilder.addedFile(spec, file); } @@ -215,14 +207,7 @@ private List writeNewManifests() throws IOException { } if (newManifests == null && !newFiles.isEmpty()) { - RollingManifestWriter writer = newRollingManifestWriter(spec); - try { - newFiles.forEach(writer::add); - } finally { - writer.close(); - } - - this.newManifests = writer.toManifestFiles(); + this.newManifests = writeDataManifests(newFiles, spec); hasNewFiles = false; } diff --git a/core/src/main/java/org/apache/iceberg/GenericBlobMetadata.java b/core/src/main/java/org/apache/iceberg/GenericBlobMetadata.java index 46bedfa01753..d3ac399556c3 100644 --- a/core/src/main/java/org/apache/iceberg/GenericBlobMetadata.java +++ b/core/src/main/java/org/apache/iceberg/GenericBlobMetadata.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg; +import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Objects; @@ -37,6 +38,13 @@ public static BlobMetadata from(org.apache.iceberg.puffin.BlobMetadata puffinMet puffinMetadata.properties()); } + public static List from( + Collection puffinMetadataList) { + return puffinMetadataList.stream() + .map(GenericBlobMetadata::from) + .collect(ImmutableList.toImmutableList()); + } + private final String type; private final long sourceSnapshotId; private final long sourceSnapshotSequenceNumber; diff --git a/core/src/main/java/org/apache/iceberg/GenericDataFile.java b/core/src/main/java/org/apache/iceberg/GenericDataFile.java index 8fe7ec756abf..7b99e7b60ab8 100644 --- a/core/src/main/java/org/apache/iceberg/GenericDataFile.java +++ b/core/src/main/java/org/apache/iceberg/GenericDataFile.java @@ -32,6 +32,11 @@ class GenericDataFile extends BaseFile implements DataFile { super(avroSchema); } + /** Used by internal readers to instantiate this class with a projection schema. */ + GenericDataFile(Types.StructType projection) { + super(projection); + } + GenericDataFile( int specId, String filePath, diff --git a/core/src/main/java/org/apache/iceberg/GenericDeleteFile.java b/core/src/main/java/org/apache/iceberg/GenericDeleteFile.java index 509bf4c16b03..77e0d8505af6 100644 --- a/core/src/main/java/org/apache/iceberg/GenericDeleteFile.java +++ b/core/src/main/java/org/apache/iceberg/GenericDeleteFile.java @@ -32,6 +32,11 @@ class GenericDeleteFile extends BaseFile implements DeleteFile { super(avroSchema); } + /** Used by internal readers to instantiate this class with a projection schema. */ + GenericDeleteFile(Types.StructType projection) { + super(projection); + } + GenericDeleteFile( int specId, FileContent content, diff --git a/core/src/main/java/org/apache/iceberg/GenericManifestEntry.java b/core/src/main/java/org/apache/iceberg/GenericManifestEntry.java index 959e2446c710..752f2785a947 100644 --- a/core/src/main/java/org/apache/iceberg/GenericManifestEntry.java +++ b/core/src/main/java/org/apache/iceberg/GenericManifestEntry.java @@ -38,8 +38,9 @@ class GenericManifestEntry> this.schema = schema; } - GenericManifestEntry(Types.StructType partitionType) { - this.schema = AvroSchemaUtil.convert(V1Metadata.entrySchema(partitionType), "manifest_entry"); + /** Used by internal readers to instantiate this class with a projection schema. */ + GenericManifestEntry(Types.StructType schema) { + this.schema = AvroSchemaUtil.convert(schema, "manifest_entry"); } private GenericManifestEntry(GenericManifestEntry toCopy, boolean fullCopy) { @@ -94,13 +95,17 @@ ManifestEntry wrapDelete( return this; } - /** @return the status of the file, whether EXISTING, ADDED, or DELETED */ + /** + * @return the status of the file, whether EXISTING, ADDED, or DELETED + */ @Override public Status status() { return status; } - /** @return id of the snapshot in which the file was added to the table */ + /** + * @return id of the snapshot in which the file was added to the table + */ @Override public Long snapshotId() { return snapshotId; @@ -116,7 +121,9 @@ public Long fileSequenceNumber() { return fileSequenceNumber; } - /** @return a file */ + /** + * @return a file + */ @Override public F file() { return file; diff --git a/core/src/main/java/org/apache/iceberg/GenericManifestFile.java b/core/src/main/java/org/apache/iceberg/GenericManifestFile.java index d081e0bdd568..7707c57a6905 100644 --- a/core/src/main/java/org/apache/iceberg/GenericManifestFile.java +++ b/core/src/main/java/org/apache/iceberg/GenericManifestFile.java @@ -28,21 +28,20 @@ import org.apache.avro.generic.IndexedRecord; import org.apache.avro.specific.SpecificData.SchemaConstructable; import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.avro.SupportsIndexProjection; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.base.Objects; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Types; import org.apache.iceberg.util.ByteBuffers; -public class GenericManifestFile +public class GenericManifestFile extends SupportsIndexProjection implements ManifestFile, StructLike, IndexedRecord, SchemaConstructable, Serializable { private static final Schema AVRO_SCHEMA = AvroSchemaUtil.convert(ManifestFile.schema(), "manifest_file"); private static final ManifestContent[] MANIFEST_CONTENT_VALUES = ManifestContent.values(); private transient Schema avroSchema; // not final for Java serialization - private int[] fromProjectionPos; // data fields private InputFile file = null; @@ -64,28 +63,12 @@ public class GenericManifestFile /** Used by Avro reflection to instantiate this class when reading manifest files. */ public GenericManifestFile(Schema avroSchema) { + super(ManifestFile.schema().asStruct(), AvroSchemaUtil.convert(avroSchema).asStructType()); this.avroSchema = avroSchema; - - List fields = AvroSchemaUtil.convert(avroSchema).asStructType().fields(); - List allFields = ManifestFile.schema().asStruct().fields(); - - this.fromProjectionPos = new int[fields.size()]; - for (int i = 0; i < fromProjectionPos.length; i += 1) { - boolean found = false; - for (int j = 0; j < allFields.size(); j += 1) { - if (fields.get(i).fieldId() == allFields.get(j).fieldId()) { - found = true; - fromProjectionPos[i] = j; - } - } - - if (!found) { - throw new IllegalArgumentException("Cannot find projected field: " + fields.get(i)); - } - } } GenericManifestFile(InputFile file, int specId) { + super(ManifestFile.schema().columns().size()); this.avroSchema = AVRO_SCHEMA; this.file = file; this.manifestPath = file.location(); @@ -101,7 +84,6 @@ public GenericManifestFile(Schema avroSchema) { this.deletedFilesCount = null; this.deletedRowsCount = null; this.partitions = null; - this.fromProjectionPos = null; this.keyMetadata = null; } @@ -122,6 +104,7 @@ public GenericManifestFile(Schema avroSchema) { Long existingRowsCount, Integer deletedFilesCount, Long deletedRowsCount) { + super(ManifestFile.schema().columns().size()); this.avroSchema = AVRO_SCHEMA; this.manifestPath = path; this.length = length; @@ -137,7 +120,6 @@ public GenericManifestFile(Schema avroSchema) { this.deletedFilesCount = deletedFilesCount; this.deletedRowsCount = deletedRowsCount; this.partitions = partitions == null ? null : partitions.toArray(new PartitionFieldSummary[0]); - this.fromProjectionPos = null; this.keyMetadata = ByteBuffers.toByteArray(keyMetadata); } @@ -157,6 +139,7 @@ public GenericManifestFile( long deletedRowsCount, List partitions, ByteBuffer keyMetadata) { + super(ManifestFile.schema().columns().size()); this.avroSchema = AVRO_SCHEMA; this.manifestPath = path; this.length = length; @@ -172,7 +155,6 @@ public GenericManifestFile( this.deletedFilesCount = deletedFilesCount; this.deletedRowsCount = deletedRowsCount; this.partitions = partitions == null ? null : partitions.toArray(new PartitionFieldSummary[0]); - this.fromProjectionPos = null; this.keyMetadata = ByteBuffers.toByteArray(keyMetadata); } @@ -182,6 +164,7 @@ public GenericManifestFile( * @param toCopy a generic manifest file to copy. */ private GenericManifestFile(GenericManifestFile toCopy) { + super(toCopy); this.avroSchema = toCopy.avroSchema; this.manifestPath = toCopy.manifestPath; this.length = toCopy.length; @@ -204,7 +187,6 @@ private GenericManifestFile(GenericManifestFile toCopy) { } else { this.partitions = null; } - this.fromProjectionPos = toCopy.fromProjectionPos; this.keyMetadata = toCopy.keyMetadata == null ? null @@ -212,7 +194,9 @@ private GenericManifestFile(GenericManifestFile toCopy) { } /** Constructor for Java serialization. */ - GenericManifestFile() {} + GenericManifestFile() { + super(ManifestFile.schema().columns().size()); + } @Override public String path() { @@ -308,18 +292,17 @@ public int size() { } @Override - public T get(int pos, Class javaClass) { - return javaClass.cast(get(pos)); + public Object get(int pos) { + return internalGet(pos, Object.class); } @Override - public Object get(int i) { - int pos = i; - // if the schema was projected, map the incoming ordinal to the expected one - if (fromProjectionPos != null) { - pos = fromProjectionPos[i]; - } - switch (pos) { + protected T internalGet(int pos, Class javaClass) { + return javaClass.cast(getByPos(pos)); + } + + private Object getByPos(int basePos) { + switch (basePos) { case 0: return manifestPath; case 1: @@ -351,19 +334,13 @@ public Object get(int i) { case 14: return keyMetadata(); default: - throw new UnsupportedOperationException("Unknown field ordinal: " + pos); + throw new UnsupportedOperationException("Unknown field ordinal: " + basePos); } } @Override - @SuppressWarnings("unchecked") - public void set(int i, T value) { - int pos = i; - // if the schema was projected, map the incoming ordinal to the expected one - if (fromProjectionPos != null) { - pos = fromProjectionPos[i]; - } - switch (pos) { + protected void internalSet(int basePos, T value) { + switch (basePos) { case 0: // always coerce to String for Serializable this.manifestPath = value.toString(); diff --git a/core/src/main/java/org/apache/iceberg/InheritableMetadataFactory.java b/core/src/main/java/org/apache/iceberg/InheritableMetadataFactory.java index 26b09833a0c0..e01346a76f3b 100644 --- a/core/src/main/java/org/apache/iceberg/InheritableMetadataFactory.java +++ b/core/src/main/java/org/apache/iceberg/InheritableMetadataFactory.java @@ -35,7 +35,10 @@ static InheritableMetadata fromManifest(ManifestFile manifest) { manifest.snapshotId() != null, "Cannot read from ManifestFile with null (unassigned) snapshot ID"); return new BaseInheritableMetadata( - manifest.partitionSpecId(), manifest.snapshotId(), manifest.sequenceNumber()); + manifest.partitionSpecId(), + manifest.snapshotId(), + manifest.sequenceNumber(), + manifest.path()); } static InheritableMetadata forCopy(long snapshotId) { @@ -46,11 +49,14 @@ static class BaseInheritableMetadata implements InheritableMetadata { private final int specId; private final long snapshotId; private final long sequenceNumber; + private final String manifestLocation; - private BaseInheritableMetadata(int specId, long snapshotId, long sequenceNumber) { + private BaseInheritableMetadata( + int specId, long snapshotId, long sequenceNumber, String manifestLocation) { this.specId = specId; this.snapshotId = snapshotId; this.sequenceNumber = sequenceNumber; + this.manifestLocation = manifestLocation; } @Override @@ -78,6 +84,7 @@ public > ManifestEntry apply(ManifestEntry manife file.setSpecId(specId); file.setDataSequenceNumber(manifestEntry.dataSequenceNumber()); file.setFileSequenceNumber(manifestEntry.fileSequenceNumber()); + file.setManifestLocation(manifestLocation); } return manifestEntry; diff --git a/core/src/main/java/org/apache/iceberg/LocationProviders.java b/core/src/main/java/org/apache/iceberg/LocationProviders.java index 0c4924b6d5a6..68bec2f4e4fc 100644 --- a/core/src/main/java/org/apache/iceberg/LocationProviders.java +++ b/core/src/main/java/org/apache/iceberg/LocationProviders.java @@ -27,7 +27,6 @@ import org.apache.iceberg.relocated.com.google.common.hash.HashCode; import org.apache.iceberg.relocated.com.google.common.hash.HashFunction; import org.apache.iceberg.relocated.com.google.common.hash.Hashing; -import org.apache.iceberg.relocated.com.google.common.io.BaseEncoding; import org.apache.iceberg.util.LocationUtil; import org.apache.iceberg.util.PropertyUtil; @@ -108,10 +107,15 @@ public String newDataLocation(String filename) { static class ObjectStoreLocationProvider implements LocationProvider { private static final HashFunction HASH_FUNC = Hashing.murmur3_32_fixed(); - private static final BaseEncoding BASE64_ENCODER = BaseEncoding.base64Url().omitPadding(); - private static final ThreadLocal TEMP = ThreadLocal.withInitial(() -> new byte[4]); + // Length of entropy generated in the file location + private static final int HASH_BINARY_STRING_BITS = 20; + // Entropy generated will be divided into dirs with this lengths + private static final int ENTROPY_DIR_LENGTH = 4; + // Will create DEPTH many dirs from the entropy + private static final int ENTROPY_DIR_DEPTH = 3; private final String storageLocation; private final String context; + private final boolean includePartitionPaths; ObjectStoreLocationProvider(String tableLocation, Map properties) { this.storageLocation = @@ -123,6 +127,11 @@ static class ObjectStoreLocationProvider implements LocationProvider { } else { this.context = pathContext(tableLocation); } + this.includePartitionPaths = + PropertyUtil.propertyAsBoolean( + properties, + TableProperties.WRITE_OBJECT_STORE_PARTITIONED_PATHS, + TableProperties.WRITE_OBJECT_STORE_PARTITIONED_PATHS_DEFAULT); } private static String dataLocation(Map properties, String tableLocation) { @@ -141,7 +150,12 @@ private static String dataLocation(Map properties, String tableL @Override public String newDataLocation(PartitionSpec spec, StructLike partitionData, String filename) { - return newDataLocation(String.format("%s/%s", spec.partitionToPath(partitionData), filename)); + if (includePartitionPaths) { + return newDataLocation( + String.format("%s/%s", spec.partitionToPath(partitionData), filename)); + } else { + return newDataLocation(filename); + } } @Override @@ -150,7 +164,13 @@ public String newDataLocation(String filename) { if (context != null) { return String.format("%s/%s/%s/%s", storageLocation, hash, context, filename); } else { - return String.format("%s/%s/%s", storageLocation, hash, filename); + // if partition paths are included, add last part of entropy as dir before partition names + if (includePartitionPaths) { + return String.format("%s/%s/%s", storageLocation, hash, filename); + } else { + // if partition paths are not included, append last part of entropy with `-` to file name + return String.format("%s/%s-%s", storageLocation, hash, filename); + } } } @@ -172,10 +192,41 @@ private static String pathContext(String tableLocation) { } private String computeHash(String fileName) { - byte[] bytes = TEMP.get(); - HashCode hash = HASH_FUNC.hashString(fileName, StandardCharsets.UTF_8); - hash.writeBytesTo(bytes, 0, 4); - return BASE64_ENCODER.encode(bytes); + HashCode hashCode = HASH_FUNC.hashString(fileName, StandardCharsets.UTF_8); + + // {@link Integer#toBinaryString} excludes leading zeros, which we want to preserve. + // force the first bit to be set to get around that. + String hashAsBinaryString = Integer.toBinaryString(hashCode.asInt() | Integer.MIN_VALUE); + // Limit hash length to HASH_BINARY_STRING_BITS + String hash = + hashAsBinaryString.substring(hashAsBinaryString.length() - HASH_BINARY_STRING_BITS); + return dirsFromHash(hash); + } + + /** + * Divides hash into directories for optimized orphan removal operation using ENTROPY_DIR_DEPTH + * and ENTROPY_DIR_LENGTH + * + * @param hash 10011001100110011001 + * @return 1001/1001/1001/10011001 with depth 3 and length 4 + */ + private String dirsFromHash(String hash) { + StringBuilder hashWithDirs = new StringBuilder(); + + for (int i = 0; i < ENTROPY_DIR_DEPTH * ENTROPY_DIR_LENGTH; i += ENTROPY_DIR_LENGTH) { + if (i > 0) { + hashWithDirs.append("/"); + } + hashWithDirs.append(hash, i, Math.min(i + ENTROPY_DIR_LENGTH, hash.length())); + } + + if (hash.length() > ENTROPY_DIR_DEPTH * ENTROPY_DIR_LENGTH) { + hashWithDirs + .append("/") + .append(hash, ENTROPY_DIR_DEPTH * ENTROPY_DIR_LENGTH, hash.length()); + } + + return hashWithDirs.toString(); } } } diff --git a/core/src/main/java/org/apache/iceberg/ManifestEntry.java b/core/src/main/java/org/apache/iceberg/ManifestEntry.java index 3eeb5407ab20..4dce92cf5c2f 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestEntry.java +++ b/core/src/main/java/org/apache/iceberg/ManifestEntry.java @@ -48,6 +48,7 @@ public int id() { Types.NestedField FILE_SEQUENCE_NUMBER = optional(4, "file_sequence_number", Types.LongType.get()); int DATA_FILE_ID = 2; + // next ID to assign: 5 static Schema getSchema(StructType partitionType) { diff --git a/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java b/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java index 3b4c5e57e31e..adc5e1b7d3c2 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java +++ b/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java @@ -25,6 +25,7 @@ import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.function.Supplier; +import java.util.stream.Collectors; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expression; @@ -39,9 +40,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.CharSequenceSet; -import org.apache.iceberg.util.CharSequenceWrapper; import org.apache.iceberg.util.ManifestFileUtil; import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.PartitionSet; @@ -69,11 +68,11 @@ public String partition() { private final Map specsById; private final PartitionSet deleteFilePartitions; + private final Set deleteFiles = newFileSet(); private final PartitionSet dropPartitions; private final CharSequenceSet deletePaths = CharSequenceSet.empty(); private Expression deleteExpression = Expressions.alwaysFalse(); private long minSequenceNumber = 0; - private boolean hasPathOnlyDeletes = false; private boolean failAnyDelete = false; private boolean failMissingDeletePaths = false; private int duplicateDeleteCount = 0; @@ -102,6 +101,8 @@ protected ManifestFilterManager( protected abstract ManifestReader newManifestReader(ManifestFile manifest); + protected abstract Set newFileSet(); + protected void failAnyDelete() { this.failAnyDelete = true; } @@ -153,7 +154,7 @@ void caseSensitive(boolean newCaseSensitive) { void delete(F file) { Preconditions.checkNotNull(file, "Cannot delete file: null"); invalidateFilteredCache(); - deletePaths.add(file.path()); + deleteFiles.add(file); deleteFilePartitions.add(file.specId(), file.partition()); } @@ -161,12 +162,12 @@ void delete(F file) { void delete(CharSequence path) { Preconditions.checkNotNull(path, "Cannot delete file path: null"); invalidateFilteredCache(); - this.hasPathOnlyDeletes = true; deletePaths.add(path); } boolean containsDeletes() { return !deletePaths.isEmpty() + || !deleteFiles.isEmpty() || deleteExpression != Expressions.alwaysFalse() || !dropPartitions.isEmpty(); } @@ -233,23 +234,37 @@ SnapshotSummary.Builder buildSummary(Iterable manifests) { @SuppressWarnings("CollectionUndefinedEquality") private void validateRequiredDeletes(ManifestFile... manifests) { if (failMissingDeletePaths) { - CharSequenceSet deletedFiles = deletedFiles(manifests); + Set deletedFiles = deletedFiles(manifests); ValidationException.check( - deletedFiles.containsAll(deletePaths), + deletedFiles.containsAll(deleteFiles), "Missing required files to delete: %s", - COMMA.join(Iterables.filter(deletePaths, path -> !deletedFiles.contains(path)))); + COMMA.join( + deleteFiles.stream() + .filter(f -> !deletedFiles.contains(f)) + .map(ContentFile::location) + .collect(Collectors.toList()))); + + CharSequenceSet deletedFilePaths = + deletedFiles.stream() + .map(ContentFile::path) + .collect(Collectors.toCollection(CharSequenceSet::empty)); + + ValidationException.check( + deletedFilePaths.containsAll(deletePaths), + "Missing required files to delete: %s", + COMMA.join(Iterables.filter(deletePaths, path -> !deletedFilePaths.contains(path)))); } } - private CharSequenceSet deletedFiles(ManifestFile[] manifests) { - CharSequenceSet deletedFiles = CharSequenceSet.empty(); + private Set deletedFiles(ManifestFile[] manifests) { + Set deletedFiles = newFileSet(); if (manifests != null) { for (ManifestFile manifest : manifests) { Iterable manifestDeletes = filteredManifestToDeletedFiles.get(manifest); if (manifestDeletes != null) { for (F file : manifestDeletes) { - deletedFiles.add(file.path()); + deletedFiles.add(file); } } } @@ -289,7 +304,9 @@ private void invalidateFilteredCache() { cleanUncommitted(SnapshotProducer.EMPTY_SET); } - /** @return a ManifestReader that is a filtered version of the input manifest. */ + /** + * @return a ManifestReader that is a filtered version of the input manifest. + */ private ManifestFile filterManifest(Schema tableSchema, ManifestFile manifest) { ManifestFile cached = filteredManifests.get(manifest); if (cached != null) { @@ -343,9 +360,9 @@ private boolean canContainDeletedFiles(ManifestFile manifest) { } boolean canContainDroppedFiles; - if (hasPathOnlyDeletes) { + if (!deletePaths.isEmpty()) { canContainDroppedFiles = true; - } else if (!deletePaths.isEmpty()) { + } else if (!deleteFiles.isEmpty()) { // because there were no path-only deletes, the set of deleted file partitions is valid canContainDroppedFiles = ManifestFileUtil.canContainAny(manifest, deleteFilePartitions, specsById); @@ -372,6 +389,7 @@ private boolean manifestHasDeletedFiles( F file = entry.file(); boolean markedForDelete = deletePaths.contains(file.path()) + || deleteFiles.contains(file) || dropPartitions.contains(file.specId(), file.partition()) || (isDelete && entry.isLive() @@ -385,7 +403,7 @@ private boolean manifestHasDeletedFiles( || isDelete, // ignore delete files where some records may not match the expression "Cannot delete file where some, but not all, rows match filter %s: %s", this.deleteExpression, - file.path()); + file.location()); if (allRowsMatch) { if (failAnyDelete) { @@ -407,8 +425,7 @@ private ManifestFile filterManifestWithDeletedFiles( boolean isDelete = reader.isDeleteManifestReader(); // when this point is reached, there is at least one file that will be deleted in the // manifest. produce a copy of the manifest with all deleted files removed. - List deletedFiles = Lists.newArrayList(); - Set deletedPaths = Sets.newHashSet(); + Set deletedFiles = newFileSet(); try { ManifestWriter writer = newManifestWriter(reader.spec()); @@ -420,6 +437,7 @@ private ManifestFile filterManifestWithDeletedFiles( F file = entry.file(); boolean markedForDelete = deletePaths.contains(file.path()) + || deleteFiles.contains(file) || dropPartitions.contains(file.specId(), file.partition()) || (isDelete && entry.isLive() @@ -434,23 +452,21 @@ private ManifestFile filterManifestWithDeletedFiles( // the expression "Cannot delete file where some, but not all, rows match filter %s: %s", this.deleteExpression, - file.path()); + file.location()); if (allRowsMatch) { writer.delete(entry); - CharSequenceWrapper wrapper = CharSequenceWrapper.wrap(entry.file().path()); - if (deletedPaths.contains(wrapper)) { + if (deletedFiles.contains(file)) { LOG.warn( "Deleting a duplicate path from manifest {}: {}", manifest.path(), - wrapper.get()); + file.location()); duplicateDeleteCount += 1; } else { // only add the file to deletes if it is a new delete // this keeps the snapshot summary accurate for non-duplicate data - deletedFiles.add(entry.file().copyWithoutStats()); - deletedPaths.add(wrapper); + deletedFiles.add(file.copyWithoutStats()); } } else { writer.existing(entry); diff --git a/core/src/main/java/org/apache/iceberg/ManifestGroup.java b/core/src/main/java/org/apache/iceberg/ManifestGroup.java index 38ad2661a8a3..10cf6bd3bfbd 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestGroup.java +++ b/core/src/main/java/org/apache/iceberg/ManifestGroup.java @@ -184,7 +184,7 @@ public CloseableIterable plan(CreateTasksFunction cre DeleteFileIndex deleteFiles = deleteIndexBuilder.scanMetrics(scanMetrics).build(); boolean dropStats = ManifestReader.dropStats(columns); - if (!deleteFiles.isEmpty()) { + if (deleteFiles.hasEqualityDeletes()) { select(ManifestReader.withStatsColumns(columns)); } diff --git a/core/src/main/java/org/apache/iceberg/ManifestReader.java b/core/src/main/java/org/apache/iceberg/ManifestReader.java index b5f85813dd2f..cf04eb7c472a 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestReader.java +++ b/core/src/main/java/org/apache/iceberg/ManifestReader.java @@ -25,8 +25,10 @@ import java.util.List; import java.util.Map; import java.util.Set; +import org.apache.avro.io.DatumReader; import org.apache.iceberg.avro.Avro; import org.apache.iceberg.avro.AvroIterable; +import org.apache.iceberg.avro.InternalReader; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.expressions.Evaluator; import org.apache.iceberg.expressions.Expression; @@ -65,16 +67,16 @@ public class ManifestReader> extends CloseableGroup "record_count"); protected enum FileType { - DATA_FILES(GenericDataFile.class.getName()), - DELETE_FILES(GenericDeleteFile.class.getName()); + DATA_FILES(GenericDataFile.class), + DELETE_FILES(GenericDeleteFile.class); - private final String fileClass; + private final Class fileClass; - FileType(String fileClass) { + FileType(Class fileClass) { this.fileClass = fileClass; } - private String fileClass() { + private Class fileClass() { return fileClass; } } @@ -261,12 +263,7 @@ private CloseableIterable> open(Schema projection) { AvroIterable> reader = Avro.read(file) .project(ManifestEntry.wrapFileSchema(Types.StructType.of(fields))) - .rename("manifest_entry", GenericManifestEntry.class.getName()) - .rename("partition", PartitionData.class.getName()) - .rename("r102", PartitionData.class.getName()) - .rename("data_file", content.fileClass()) - .rename("r2", content.fileClass()) - .classLoader(GenericManifestEntry.class.getClassLoader()) + .createResolvingReader(this::newReader) .reuseContainers() .build(); @@ -279,6 +276,13 @@ private CloseableIterable> open(Schema projection) { } } + private DatumReader newReader(Schema schema) { + return InternalReader.create(schema) + .setRootType(GenericManifestEntry.class) + .setCustomType(ManifestEntry.DATA_FILE_ID, content.fileClass()) + .setCustomType(DataFile.PARTITION_ID, PartitionData.class); + } + CloseableIterable> liveEntries() { return entries(true /* only live entries */); } @@ -292,7 +296,9 @@ private boolean isLiveEntry(ManifestEntry entry) { return entry != null && entry.status() != ManifestEntry.Status.DELETED; } - /** @return an Iterator of DataFile. Makes defensive copies of files before returning */ + /** + * @return an Iterator of DataFile. Makes defensive copies of files before returning + */ @Override public CloseableIterator iterator() { boolean dropStats = dropStats(columns); diff --git a/core/src/main/java/org/apache/iceberg/ManifestWriter.java b/core/src/main/java/org/apache/iceberg/ManifestWriter.java index 88587a1ebc89..fbfc62b94fe4 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestWriter.java +++ b/core/src/main/java/org/apache/iceberg/ManifestWriter.java @@ -60,7 +60,8 @@ private ManifestWriter(PartitionSpec spec, EncryptedOutputFile file, Long snapsh this.specId = spec.specId(); this.writer = newAppender(spec, this.file); this.snapshotId = snapshotId; - this.reused = new GenericManifestEntry<>(spec.partitionType()); + this.reused = + new GenericManifestEntry<>(V1Metadata.entrySchema(spec.partitionType()).asStruct()); this.stats = new PartitionSummary(spec); this.keyMetadataBuffer = (file.keyMetadata() == null) ? null : file.keyMetadata().buffer(); } diff --git a/core/src/main/java/org/apache/iceberg/MergeAppend.java b/core/src/main/java/org/apache/iceberg/MergeAppend.java index 3ef553ba7832..231ad8cc5d06 100644 --- a/core/src/main/java/org/apache/iceberg/MergeAppend.java +++ b/core/src/main/java/org/apache/iceberg/MergeAppend.java @@ -18,15 +18,9 @@ */ package org.apache.iceberg; -import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -/** - * Append implementation that produces a minimal number of manifest files. - * - *

This implementation will attempt to commit 5 times before throwing {@link - * CommitFailedException}. - */ +/** {@link AppendFiles Append} implementation that produces a minimal number of manifest files. */ class MergeAppend extends MergingSnapshotProducer implements AppendFiles { MergeAppend(String tableName, TableOperations ops) { super(tableName, ops); diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index b4c0567ab73a..50885dbb06c7 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -30,10 +30,8 @@ import java.util.Map; import java.util.Objects; import java.util.Set; -import java.util.stream.Collectors; import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.events.CreateSnapshotEvent; -import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; @@ -43,7 +41,6 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.base.Predicate; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Iterators; @@ -51,6 +48,8 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.CharSequenceSet; +import org.apache.iceberg.util.DataFileSet; +import org.apache.iceberg.util.DeleteFileSet; import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.PartitionSet; import org.apache.iceberg.util.SnapshotUtil; @@ -81,11 +80,9 @@ abstract class MergingSnapshotProducer extends SnapshotProducer { private final ManifestFilterManager deleteFilterManager; // update data - private final Map> newDataFilesBySpec = Maps.newHashMap(); - private final CharSequenceSet newDataFilePaths = CharSequenceSet.empty(); - private final CharSequenceSet newDeleteFilePaths = CharSequenceSet.empty(); + private final Map newDataFilesBySpec = Maps.newHashMap(); private Long newDataFilesDataSequenceNumber; - private final Map> newDeleteFilesBySpec = Maps.newHashMap(); + private final Map newDeleteFilesBySpec = Maps.newHashMap(); private final List appendManifests = Lists.newArrayList(); private final List rewrittenAppendManifests = Lists.newArrayList(); private final SnapshotSummary.Builder addedFilesSummary = SnapshotSummary.builder(); @@ -141,18 +138,13 @@ protected boolean isCaseSensitive() { } protected PartitionSpec dataSpec() { - Set specs = dataSpecs(); + Set specIds = newDataFilesBySpec.keySet(); Preconditions.checkState( - specs.size() == 1, - "Cannot return a single partition spec: data files with different partition specs have been added"); - return specs.iterator().next(); - } - - protected Set dataSpecs() { - Set specs = newDataFilesBySpec.keySet(); + !specIds.isEmpty(), "Cannot determine partition specs: no data files have been added"); Preconditions.checkState( - !specs.isEmpty(), "Cannot determine partition specs: no data files have been added"); - return ImmutableSet.copyOf(specs); + specIds.size() == 1, + "Cannot return a single partition spec: data files with different partition specs have been added"); + return spec(Iterables.getOnlyElement(specIds)); } protected Expression rowFilter() { @@ -160,12 +152,9 @@ protected Expression rowFilter() { } protected List addedDataFiles() { - return ImmutableList.copyOf( - newDataFilesBySpec.values().stream().flatMap(List::stream).collect(Collectors.toList())); - } - - protected Map> addedDataFilesBySpec() { - return ImmutableMap.copyOf(newDataFilesBySpec); + return newDataFilesBySpec.values().stream() + .flatMap(Set::stream) + .collect(ImmutableList.toImmutableList()); } protected void failAnyDelete() { @@ -235,43 +224,49 @@ protected boolean addsDeleteFiles() { /** Add a data file to the new snapshot. */ protected void add(DataFile file) { Preconditions.checkNotNull(file, "Invalid data file: null"); - if (newDataFilePaths.add(file.path())) { - PartitionSpec fileSpec = ops.current().spec(file.specId()); - Preconditions.checkArgument( - fileSpec != null, - "Cannot find partition spec %s for data file: %s", - file.specId(), - file.path()); - - addedFilesSummary.addedFile(fileSpec, file); + PartitionSpec spec = spec(file.specId()); + Preconditions.checkArgument( + spec != null, + "Cannot find partition spec %s for data file: %s", + file.specId(), + file.location()); + + DataFileSet dataFiles = + newDataFilesBySpec.computeIfAbsent(spec.specId(), ignored -> DataFileSet.create()); + if (dataFiles.add(file)) { + addedFilesSummary.addedFile(spec, file); hasNewDataFiles = true; - List newDataFiles = - newDataFilesBySpec.computeIfAbsent(fileSpec, ignored -> Lists.newArrayList()); - newDataFiles.add(file); } } + private PartitionSpec spec(int specId) { + return ops.current().spec(specId); + } + /** Add a delete file to the new snapshot. */ protected void add(DeleteFile file) { Preconditions.checkNotNull(file, "Invalid delete file: null"); - add(new DeleteFileHolder(file)); + add(new PendingDeleteFile(file)); } /** Add a delete file to the new snapshot. */ protected void add(DeleteFile file, long dataSequenceNumber) { Preconditions.checkNotNull(file, "Invalid delete file: null"); - add(new DeleteFileHolder(file, dataSequenceNumber)); + add(new PendingDeleteFile(file, dataSequenceNumber)); } - private void add(DeleteFileHolder fileHolder) { - int specId = fileHolder.deleteFile().specId(); - PartitionSpec fileSpec = ops.current().spec(specId); - List deleteFiles = - newDeleteFilesBySpec.computeIfAbsent(specId, s -> Lists.newArrayList()); - - if (newDeleteFilePaths.add(fileHolder.deleteFile().path())) { - deleteFiles.add(fileHolder); - addedFilesSummary.addedFile(fileSpec, fileHolder.deleteFile()); + private void add(PendingDeleteFile file) { + PartitionSpec spec = spec(file.specId()); + Preconditions.checkArgument( + spec != null, + "Cannot find partition spec %s for delete file: %s", + file.specId(), + file.location()); + + DeleteFileSet deleteFiles = + newDeleteFilesBySpec.computeIfAbsent(spec.specId(), ignored -> DeleteFileSet.create()); + if (deleteFiles.add(file)) { + addedFilesSummary.addedFile(spec, file); hasNewDeleteFiles = true; } } @@ -971,24 +966,12 @@ private List newDataFilesAsManifests() { if (cachedNewDataManifests.isEmpty()) { newDataFilesBySpec.forEach( - (dataSpec, newDataFiles) -> { - try { - RollingManifestWriter writer = newRollingManifestWriter(dataSpec); - try { - if (newDataFilesDataSequenceNumber == null) { - newDataFiles.forEach(writer::add); - } else { - newDataFiles.forEach(f -> writer.add(f, newDataFilesDataSequenceNumber)); - } - } finally { - writer.close(); - } - this.cachedNewDataManifests.addAll(writer.toManifestFiles()); - this.hasNewDataFiles = false; - } catch (IOException e) { - throw new RuntimeIOException(e, "Failed to close manifest writer"); - } + (specId, dataFiles) -> { + List newDataManifests = + writeDataManifests(dataFiles, newDataFilesDataSequenceNumber, spec(specId)); + cachedNewDataManifests.addAll(newDataManifests); }); + this.hasNewDataFiles = false; } return cachedNewDataManifests; @@ -1016,24 +999,8 @@ private List newDeleteFilesAsManifests() { newDeleteFilesBySpec.forEach( (specId, deleteFiles) -> { PartitionSpec spec = ops.current().spec(specId); - try { - RollingManifestWriter writer = newRollingDeleteManifestWriter(spec); - try { - deleteFiles.forEach( - df -> { - if (df.dataSequenceNumber() != null) { - writer.add(df.deleteFile(), df.dataSequenceNumber()); - } else { - writer.add(df.deleteFile()); - } - }); - } finally { - writer.close(); - } - cachedNewDeleteManifests.addAll(writer.toManifestFiles()); - } catch (IOException e) { - throw new RuntimeIOException(e, "Failed to close manifest writer"); - } + List newDeleteManifests = writeDeleteManifests(deleteFiles, spec); + cachedNewDeleteManifests.addAll(newDeleteManifests); }); this.hasNewDeleteFiles = false; @@ -1061,6 +1028,11 @@ protected ManifestWriter newManifestWriter(PartitionSpec manifestSpec) protected ManifestReader newManifestReader(ManifestFile manifest) { return MergingSnapshotProducer.this.newManifestReader(manifest); } + + @Override + protected Set newFileSet() { + return DataFileSet.create(); + } } private class DataFileMergeManager extends ManifestMergeManager { @@ -1114,6 +1086,11 @@ protected ManifestWriter newManifestWriter(PartitionSpec manifestSpe protected ManifestReader newManifestReader(ManifestFile manifest) { return MergingSnapshotProducer.this.newDeleteManifestReader(manifest); } + + @Override + protected Set newFileSet() { + return DeleteFileSet.create(); + } } private class DeleteFileMergeManager extends ManifestMergeManager { @@ -1147,38 +1124,4 @@ protected ManifestReader newManifestReader(ManifestFile manifest) { return MergingSnapshotProducer.this.newDeleteManifestReader(manifest); } } - - private static class DeleteFileHolder { - private final DeleteFile deleteFile; - private final Long dataSequenceNumber; - - /** - * Wrap a delete file for commit with a given data sequence number - * - * @param deleteFile delete file - * @param dataSequenceNumber data sequence number to apply - */ - DeleteFileHolder(DeleteFile deleteFile, long dataSequenceNumber) { - this.deleteFile = deleteFile; - this.dataSequenceNumber = dataSequenceNumber; - } - - /** - * Wrap a delete file for commit with the latest sequence number - * - * @param deleteFile delete file - */ - DeleteFileHolder(DeleteFile deleteFile) { - this.deleteFile = deleteFile; - this.dataSequenceNumber = null; - } - - public DeleteFile deleteFile() { - return deleteFile; - } - - public Long dataSequenceNumber() { - return dataSequenceNumber; - } - } } diff --git a/core/src/main/java/org/apache/iceberg/MetricsModes.java b/core/src/main/java/org/apache/iceberg/MetricsModes.java index 3dc2434310a5..cd5d5eab1273 100644 --- a/core/src/main/java/org/apache/iceberg/MetricsModes.java +++ b/core/src/main/java/org/apache/iceberg/MetricsModes.java @@ -114,7 +114,7 @@ public int length() { @Override public String toString() { - return String.format("truncate(%d)", length); + return String.format(Locale.ROOT, "truncate(%d)", length); } @Override diff --git a/core/src/main/java/org/apache/iceberg/PartitionStats.java b/core/src/main/java/org/apache/iceberg/PartitionStats.java new file mode 100644 index 000000000000..e4cbd1f6b9bd --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/PartitionStats.java @@ -0,0 +1,252 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +public class PartitionStats implements StructLike { + + private static final int STATS_COUNT = 12; + + private StructLike partition; + private int specId; + private long dataRecordCount; + private int dataFileCount; + private long totalDataFileSizeInBytes; + private long positionDeleteRecordCount; + private int positionDeleteFileCount; + private long equalityDeleteRecordCount; + private int equalityDeleteFileCount; + private long totalRecordCount; + private Long lastUpdatedAt; // null by default + private Long lastUpdatedSnapshotId; // null by default + + public PartitionStats(StructLike partition, int specId) { + this.partition = partition; + this.specId = specId; + } + + public StructLike partition() { + return partition; + } + + public int specId() { + return specId; + } + + public long dataRecordCount() { + return dataRecordCount; + } + + public int dataFileCount() { + return dataFileCount; + } + + public long totalDataFileSizeInBytes() { + return totalDataFileSizeInBytes; + } + + public long positionDeleteRecordCount() { + return positionDeleteRecordCount; + } + + public int positionDeleteFileCount() { + return positionDeleteFileCount; + } + + public long equalityDeleteRecordCount() { + return equalityDeleteRecordCount; + } + + public int equalityDeleteFileCount() { + return equalityDeleteFileCount; + } + + public long totalRecordCount() { + return totalRecordCount; + } + + public Long lastUpdatedAt() { + return lastUpdatedAt; + } + + public Long lastUpdatedSnapshotId() { + return lastUpdatedSnapshotId; + } + + /** + * Updates the partition stats from the data/delete file. + * + * @param file the {@link ContentFile} from the manifest entry. + * @param snapshot the snapshot corresponding to the live entry. + */ + public void liveEntry(ContentFile file, Snapshot snapshot) { + Preconditions.checkArgument(specId == file.specId(), "Spec IDs must match"); + + switch (file.content()) { + case DATA: + this.dataRecordCount += file.recordCount(); + this.dataFileCount += 1; + this.totalDataFileSizeInBytes += file.fileSizeInBytes(); + break; + case POSITION_DELETES: + this.positionDeleteRecordCount += file.recordCount(); + this.positionDeleteFileCount += 1; + break; + case EQUALITY_DELETES: + this.equalityDeleteRecordCount += file.recordCount(); + this.equalityDeleteFileCount += 1; + break; + default: + throw new UnsupportedOperationException("Unsupported file content type: " + file.content()); + } + + if (snapshot != null) { + updateSnapshotInfo(snapshot.snapshotId(), snapshot.timestampMillis()); + } + + // Note: Not computing the `TOTAL_RECORD_COUNT` for now as it needs scanning the data. + } + + /** + * Updates the modified time and snapshot ID for the deleted manifest entry. + * + * @param snapshot the snapshot corresponding to the deleted manifest entry. + */ + public void deletedEntry(Snapshot snapshot) { + if (snapshot != null) { + updateSnapshotInfo(snapshot.snapshotId(), snapshot.timestampMillis()); + } + } + + /** + * Appends statistics from given entry to current entry. + * + * @param entry the entry from which statistics will be sourced. + */ + public void appendStats(PartitionStats entry) { + Preconditions.checkArgument(specId == entry.specId(), "Spec IDs must match"); + + this.dataRecordCount += entry.dataRecordCount; + this.dataFileCount += entry.dataFileCount; + this.totalDataFileSizeInBytes += entry.totalDataFileSizeInBytes; + this.positionDeleteRecordCount += entry.positionDeleteRecordCount; + this.positionDeleteFileCount += entry.positionDeleteFileCount; + this.equalityDeleteRecordCount += entry.equalityDeleteRecordCount; + this.equalityDeleteFileCount += entry.equalityDeleteFileCount; + this.totalRecordCount += entry.totalRecordCount; + + if (entry.lastUpdatedAt != null) { + updateSnapshotInfo(entry.lastUpdatedSnapshotId, entry.lastUpdatedAt); + } + } + + private void updateSnapshotInfo(long snapshotId, long updatedAt) { + if (lastUpdatedAt == null || lastUpdatedAt < updatedAt) { + this.lastUpdatedAt = updatedAt; + this.lastUpdatedSnapshotId = snapshotId; + } + } + + @Override + public int size() { + return STATS_COUNT; + } + + @Override + public T get(int pos, Class javaClass) { + switch (pos) { + case 0: + return javaClass.cast(partition); + case 1: + return javaClass.cast(specId); + case 2: + return javaClass.cast(dataRecordCount); + case 3: + return javaClass.cast(dataFileCount); + case 4: + return javaClass.cast(totalDataFileSizeInBytes); + case 5: + return javaClass.cast(positionDeleteRecordCount); + case 6: + return javaClass.cast(positionDeleteFileCount); + case 7: + return javaClass.cast(equalityDeleteRecordCount); + case 8: + return javaClass.cast(equalityDeleteFileCount); + case 9: + return javaClass.cast(totalRecordCount); + case 10: + return javaClass.cast(lastUpdatedAt); + case 11: + return javaClass.cast(lastUpdatedSnapshotId); + default: + throw new UnsupportedOperationException("Unknown position: " + pos); + } + } + + @Override + public void set(int pos, T value) { + switch (pos) { + case 0: + this.partition = (StructLike) value; + break; + case 1: + this.specId = (int) value; + break; + case 2: + this.dataRecordCount = (long) value; + break; + case 3: + this.dataFileCount = (int) value; + break; + case 4: + this.totalDataFileSizeInBytes = (long) value; + break; + case 5: + // optional field as per spec, implementation initialize to 0 for counters + this.positionDeleteRecordCount = value == null ? 0L : (long) value; + break; + case 6: + // optional field as per spec, implementation initialize to 0 for counters + this.positionDeleteFileCount = value == null ? 0 : (int) value; + break; + case 7: + // optional field as per spec, implementation initialize to 0 for counters + this.equalityDeleteRecordCount = value == null ? 0L : (long) value; + break; + case 8: + // optional field as per spec, implementation initialize to 0 for counters + this.equalityDeleteFileCount = value == null ? 0 : (int) value; + break; + case 9: + // optional field as per spec, implementation initialize to 0 for counters + this.totalRecordCount = value == null ? 0L : (long) value; + break; + case 10: + this.lastUpdatedAt = (Long) value; + break; + case 11: + this.lastUpdatedSnapshotId = (Long) value; + break; + default: + throw new UnsupportedOperationException("Unknown position: " + pos); + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/PartitionStatsUtil.java b/core/src/main/java/org/apache/iceberg/PartitionStatsUtil.java new file mode 100644 index 000000000000..1fe4e6767fe6 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/PartitionStatsUtil.java @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Collection; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Queue; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Queues; +import org.apache.iceberg.types.Comparators; +import org.apache.iceberg.types.Types.StructType; +import org.apache.iceberg.util.PartitionMap; +import org.apache.iceberg.util.PartitionUtil; +import org.apache.iceberg.util.Tasks; +import org.apache.iceberg.util.ThreadPools; + +public class PartitionStatsUtil { + + private PartitionStatsUtil() {} + + /** + * Computes the partition stats for the given snapshot of the table. + * + * @param table the table for which partition stats to be computed. + * @param snapshot the snapshot for which partition stats is computed. + * @return the collection of {@link PartitionStats} + */ + public static Collection computeStats(Table table, Snapshot snapshot) { + Preconditions.checkArgument(table != null, "table cannot be null"); + Preconditions.checkArgument(Partitioning.isPartitioned(table), "table must be partitioned"); + Preconditions.checkArgument(snapshot != null, "snapshot cannot be null"); + + StructType partitionType = Partitioning.partitionType(table); + List manifests = snapshot.allManifests(table.io()); + Queue> statsByManifest = Queues.newConcurrentLinkedQueue(); + Tasks.foreach(manifests) + .stopOnFailure() + .throwFailureWhenFinished() + .executeWith(ThreadPools.getWorkerPool()) + .run(manifest -> statsByManifest.add(collectStats(table, manifest, partitionType))); + + return mergeStats(statsByManifest, table.specs()); + } + + /** + * Sorts the {@link PartitionStats} based on the partition data. + * + * @param stats collection of {@link PartitionStats} which needs to be sorted. + * @param partitionType unified partition schema. + * @return the list of {@link PartitionStats} + */ + public static List sortStats( + Collection stats, StructType partitionType) { + List entries = Lists.newArrayList(stats); + entries.sort(partitionStatsCmp(partitionType)); + return entries; + } + + private static Comparator partitionStatsCmp(StructType partitionType) { + return Comparator.comparing(PartitionStats::partition, Comparators.forType(partitionType)); + } + + private static PartitionMap collectStats( + Table table, ManifestFile manifest, StructType partitionType) { + try (ManifestReader reader = openManifest(table, manifest)) { + PartitionMap statsMap = PartitionMap.create(table.specs()); + int specId = manifest.partitionSpecId(); + PartitionSpec spec = table.specs().get(specId); + PartitionData keyTemplate = new PartitionData(partitionType); + + for (ManifestEntry entry : reader.entries()) { + ContentFile file = entry.file(); + StructLike coercedPartition = + PartitionUtil.coercePartition(partitionType, spec, file.partition()); + StructLike key = keyTemplate.copyFor(coercedPartition); + Snapshot snapshot = table.snapshot(entry.snapshotId()); + PartitionStats stats = + statsMap.computeIfAbsent(specId, key, () -> new PartitionStats(key, specId)); + if (entry.isLive()) { + stats.liveEntry(file, snapshot); + } else { + stats.deletedEntry(snapshot); + } + } + + return statsMap; + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + private static ManifestReader openManifest(Table table, ManifestFile manifest) { + List projection = BaseScan.scanColumns(manifest.content()); + return ManifestFiles.open(manifest, table.io()).select(projection); + } + + private static Collection mergeStats( + Queue> statsByManifest, Map specs) { + PartitionMap statsMap = PartitionMap.create(specs); + + for (PartitionMap stats : statsByManifest) { + stats.forEach( + (key, value) -> + statsMap.merge( + key, + value, + (existingEntry, newEntry) -> { + existingEntry.appendStats(newEntry); + return existingEntry; + })); + } + + return statsMap.values(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/Partitioning.java b/core/src/main/java/org/apache/iceberg/Partitioning.java index 7e4fcae333d8..832e0b59fe50 100644 --- a/core/src/main/java/org/apache/iceberg/Partitioning.java +++ b/core/src/main/java/org/apache/iceberg/Partitioning.java @@ -242,6 +242,16 @@ public static StructType partitionType(Table table) { return buildPartitionProjectionType("table partition", specs, allFieldIds(specs)); } + /** + * Checks if any of the specs in a table is partitioned. + * + * @param table the table to check. + * @return {@code true} if the table is partitioned, {@code false} otherwise. + */ + public static boolean isPartitioned(Table table) { + return table.specs().values().stream().anyMatch(PartitionSpec::isPartitioned); + } + private static StructType buildPartitionProjectionType( String typeName, Collection specs, Set projectedFieldIds) { diff --git a/core/src/main/java/org/apache/iceberg/PartitionsTable.java b/core/src/main/java/org/apache/iceberg/PartitionsTable.java index 5ff796e95827..6d0fc8c235f9 100644 --- a/core/src/main/java/org/apache/iceberg/PartitionsTable.java +++ b/core/src/main/java/org/apache/iceberg/PartitionsTable.java @@ -201,7 +201,7 @@ private static CloseableIterable> readEntries( return CloseableIterable.transform( ManifestFiles.open(manifest, table.io(), table.specs()) .caseSensitive(scan.isCaseSensitive()) - .select(scanColumns(manifest.content())) // don't select stats columns + .select(BaseScan.scanColumns(manifest.content())) // don't select stats columns .liveEntries(), t -> (ManifestEntry>) @@ -209,17 +209,6 @@ private static CloseableIterable> readEntries( t.copyWithoutStats()); } - private static List scanColumns(ManifestContent content) { - switch (content) { - case DATA: - return BaseScan.SCAN_COLUMNS; - case DELETES: - return BaseScan.DELETE_SCAN_COLUMNS; - default: - throw new UnsupportedOperationException("Cannot read unknown manifest type: " + content); - } - } - private static CloseableIterable filteredManifests( StaticTableScan scan, Table table, List manifestFilesList) { CloseableIterable manifestFiles = diff --git a/core/src/main/java/org/apache/iceberg/RemoveSnapshots.java b/core/src/main/java/org/apache/iceberg/RemoveSnapshots.java index fa6fcdf41442..7558ea7d8a3e 100644 --- a/core/src/main/java/org/apache/iceberg/RemoveSnapshots.java +++ b/core/src/main/java/org/apache/iceberg/RemoveSnapshots.java @@ -84,6 +84,7 @@ public void accept(String file) { private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE; private ExecutorService planExecutorService = ThreadPools.getWorkerPool(); private Boolean incrementalCleanup; + private boolean specifiedSnapshotId = false; RemoveSnapshots(TableOperations ops) { this.ops = ops; @@ -116,6 +117,7 @@ public ExpireSnapshots cleanExpiredFiles(boolean clean) { public ExpireSnapshots expireSnapshotId(long expireSnapshotId) { LOG.info("Expiring snapshot with id: {}", expireSnapshotId); idsToRemove.add(expireSnapshotId); + specifiedSnapshotId = true; return this; } @@ -321,6 +323,15 @@ ExpireSnapshots withIncrementalCleanup(boolean useIncrementalCleanup) { private void cleanExpiredSnapshots() { TableMetadata current = ops.refresh(); + if (specifiedSnapshotId) { + if (incrementalCleanup != null && incrementalCleanup) { + throw new UnsupportedOperationException( + "Cannot clean files incrementally when snapshot IDs are specified"); + } + + incrementalCleanup = false; + } + if (incrementalCleanup == null) { incrementalCleanup = current.refs().size() == 1; } diff --git a/core/src/main/java/org/apache/iceberg/ScanSummary.java b/core/src/main/java/org/apache/iceberg/ScanSummary.java index f975ef1636df..1ea171c5b2c3 100644 --- a/core/src/main/java/org/apache/iceberg/ScanSummary.java +++ b/core/src/main/java/org/apache/iceberg/ScanSummary.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.Comparator; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.NavigableMap; import java.util.Set; @@ -346,7 +347,7 @@ public void update(K key, Function updateFunc) { while (map.size() > maxSize) { if (throwIfLimited) { throw new IllegalStateException( - String.format("Too many matching keys: more than %d", maxSize)); + String.format(Locale.ROOT, "Too many matching keys: more than %d", maxSize)); } this.cut = map.lastKey(); map.remove(cut); diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 74997cc89849..89f9eab7192a 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -34,15 +34,21 @@ import com.github.benmanes.caffeine.cache.Caffeine; import com.github.benmanes.caffeine.cache.LoadingCache; import java.io.IOException; +import java.math.RoundingMode; +import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.Collection; import java.util.List; +import java.util.Locale; import java.util.Map; +import java.util.Queue; import java.util.Set; import java.util.UUID; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; +import java.util.function.Function; import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.encryption.EncryptingFileIO; import org.apache.iceberg.events.CreateSnapshotEvent; @@ -59,10 +65,14 @@ import org.apache.iceberg.metrics.LoggingMetricsReporter; import org.apache.iceberg.metrics.MetricsReporter; import org.apache.iceberg.metrics.Timer.Timed; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Queues; import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.relocated.com.google.common.math.IntMath; import org.apache.iceberg.util.Exceptions; import org.apache.iceberg.util.SnapshotUtil; import org.apache.iceberg.util.Tasks; @@ -70,9 +80,16 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * Keeps common functionality to create a new snapshot. + * + *

The number of attempted commits is controlled by {@link TableProperties#COMMIT_NUM_RETRIES} + * and {@link TableProperties#COMMIT_NUM_RETRIES_DEFAULT} properties. + */ @SuppressWarnings("UnnecessaryAnonymousClass") abstract class SnapshotProducer implements SnapshotUpdate { private static final Logger LOG = LoggerFactory.getLogger(SnapshotProducer.class); + static final int MIN_FILE_GROUP_SIZE = 10_000; static final Set EMPTY_SET = Sets.newHashSet(); /** Default callback used to delete files. */ @@ -497,7 +514,11 @@ protected OutputFile manifestListPath() { ops.metadataFileLocation( FileFormat.AVRO.addExtension( String.format( - "snap-%d-%d-%s", snapshotId(), attempt.incrementAndGet(), commitUUID)))); + Locale.ROOT, + "snap-%d-%d-%s", + snapshotId(), + attempt.incrementAndGet(), + commitUUID)))); } protected EncryptedOutputFile newManifestOutputFile() { @@ -554,6 +575,91 @@ protected boolean cleanupAfterCommit() { return true; } + protected List writeDataManifests(Collection files, PartitionSpec spec) { + return writeDataManifests(files, null /* inherit data seq */, spec); + } + + protected List writeDataManifests( + Collection files, Long dataSeq, PartitionSpec spec) { + return writeManifests(files, group -> writeDataFileGroup(group, dataSeq, spec)); + } + + private List writeDataFileGroup( + Collection files, Long dataSeq, PartitionSpec spec) { + RollingManifestWriter writer = newRollingManifestWriter(spec); + + try (RollingManifestWriter closableWriter = writer) { + if (dataSeq != null) { + files.forEach(file -> closableWriter.add(file, dataSeq)); + } else { + files.forEach(closableWriter::add); + } + } catch (IOException e) { + throw new RuntimeIOException(e, "Failed to write data manifests"); + } + + return writer.toManifestFiles(); + } + + protected List writeDeleteManifests( + Collection files, PartitionSpec spec) { + return writeManifests(files, group -> writeDeleteFileGroup(group, spec)); + } + + private List writeDeleteFileGroup( + Collection files, PartitionSpec spec) { + RollingManifestWriter writer = newRollingDeleteManifestWriter(spec); + + try (RollingManifestWriter closableWriter = writer) { + for (DeleteFile file : files) { + Preconditions.checkArgument( + file instanceof PendingDeleteFile, "Invalid delete file: must be PendingDeleteFile"); + if (file.dataSequenceNumber() != null) { + closableWriter.add(file, file.dataSequenceNumber()); + } else { + closableWriter.add(file); + } + } + } catch (IOException e) { + throw new RuntimeIOException(e, "Failed to write delete manifests"); + } + + return writer.toManifestFiles(); + } + + private static List writeManifests( + Collection files, Function, List> writeFunc) { + int parallelism = manifestWriterCount(ThreadPools.WORKER_THREAD_POOL_SIZE, files.size()); + List> groups = divide(files, parallelism); + Queue manifests = Queues.newConcurrentLinkedQueue(); + Tasks.foreach(groups) + .stopOnFailure() + .throwFailureWhenFinished() + .executeWith(ThreadPools.getWorkerPool()) + .run(group -> manifests.addAll(writeFunc.apply(group))); + return ImmutableList.copyOf(manifests); + } + + private static List> divide(Collection collection, int groupCount) { + List list = Lists.newArrayList(collection); + int groupSize = IntMath.divide(list.size(), groupCount, RoundingMode.CEILING); + return Lists.partition(list, groupSize); + } + + /** + * Calculates how many manifest writers can be used to concurrently to handle the given number of + * files without creating too small manifests. + * + * @param workerPoolSize the size of the available worker pool + * @param fileCount the total number of files to be processed + * @return the number of manifest writers that can be used concurrently + */ + @VisibleForTesting + static int manifestWriterCount(int workerPoolSize, int fileCount) { + int limit = IntMath.divide(fileCount, MIN_FILE_GROUP_SIZE, RoundingMode.HALF_UP); + return Math.max(1, Math.min(workerPoolSize, limit)); + } + private static ManifestFile addMetadata(TableOperations ops, ManifestFile manifest) { try (ManifestReader reader = ManifestFiles.read(manifest, ops.io(), ops.current().specsById())) { @@ -654,4 +760,168 @@ private static void updateTotal( } } } + + protected static class PendingDeleteFile implements DeleteFile { + private final DeleteFile deleteFile; + private final Long dataSequenceNumber; + + /** + * Wrap a delete file for commit with a given data sequence number. + * + * @param deleteFile delete file + * @param dataSequenceNumber data sequence number to apply + */ + PendingDeleteFile(DeleteFile deleteFile, long dataSequenceNumber) { + this.deleteFile = deleteFile; + this.dataSequenceNumber = dataSequenceNumber; + } + + /** + * Wrap a delete file for commit with the latest sequence number. + * + * @param deleteFile delete file + */ + PendingDeleteFile(DeleteFile deleteFile) { + this.deleteFile = deleteFile; + this.dataSequenceNumber = null; + } + + private PendingDeleteFile wrap(DeleteFile file) { + if (null != dataSequenceNumber) { + return new PendingDeleteFile(file, dataSequenceNumber); + } + + return new PendingDeleteFile(file); + } + + @Override + public Long dataSequenceNumber() { + return dataSequenceNumber; + } + + @Override + public Long fileSequenceNumber() { + return deleteFile.fileSequenceNumber(); + } + + @Override + public DeleteFile copy() { + return wrap(deleteFile.copy()); + } + + @Override + public DeleteFile copyWithoutStats() { + return wrap(deleteFile.copyWithoutStats()); + } + + @Override + public DeleteFile copyWithStats(Set requestedColumnIds) { + return wrap(deleteFile.copyWithStats(requestedColumnIds)); + } + + @Override + public DeleteFile copy(boolean withStats) { + return wrap(deleteFile.copy(withStats)); + } + + @Override + public String manifestLocation() { + return deleteFile.manifestLocation(); + } + + @Override + public Long pos() { + return deleteFile.pos(); + } + + @Override + public int specId() { + return deleteFile.specId(); + } + + @Override + public FileContent content() { + return deleteFile.content(); + } + + @Override + public CharSequence path() { + return deleteFile.path(); + } + + @Override + public String location() { + return deleteFile.location(); + } + + @Override + public FileFormat format() { + return deleteFile.format(); + } + + @Override + public StructLike partition() { + return deleteFile.partition(); + } + + @Override + public long recordCount() { + return deleteFile.recordCount(); + } + + @Override + public long fileSizeInBytes() { + return deleteFile.fileSizeInBytes(); + } + + @Override + public Map columnSizes() { + return deleteFile.columnSizes(); + } + + @Override + public Map valueCounts() { + return deleteFile.valueCounts(); + } + + @Override + public Map nullValueCounts() { + return deleteFile.nullValueCounts(); + } + + @Override + public Map nanValueCounts() { + return deleteFile.nanValueCounts(); + } + + @Override + public Map lowerBounds() { + return deleteFile.lowerBounds(); + } + + @Override + public Map upperBounds() { + return deleteFile.upperBounds(); + } + + @Override + public ByteBuffer keyMetadata() { + return deleteFile.keyMetadata(); + } + + @Override + public List splitOffsets() { + return deleteFile.splitOffsets(); + } + + @Override + public List equalityFieldIds() { + return deleteFile.equalityFieldIds(); + } + + @Override + public Integer sortOrderId() { + return deleteFile.sortOrderId(); + } + } } diff --git a/core/src/main/java/org/apache/iceberg/StreamingDelete.java b/core/src/main/java/org/apache/iceberg/StreamingDelete.java index df5a11bf31c5..81621164e4af 100644 --- a/core/src/main/java/org/apache/iceberg/StreamingDelete.java +++ b/core/src/main/java/org/apache/iceberg/StreamingDelete.java @@ -18,15 +18,9 @@ */ package org.apache.iceberg; -import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.expressions.Expression; -/** - * {@link DeleteFiles Delete} implementation that avoids loading full manifests in memory. - * - *

This implementation will attempt to commit 5 times before throwing {@link - * CommitFailedException}. - */ +/** {@link DeleteFiles Delete} implementation that avoids loading full manifests in memory. */ public class StreamingDelete extends MergingSnapshotProducer implements DeleteFiles { private boolean validateFilesToDeleteExist = false; diff --git a/core/src/main/java/org/apache/iceberg/SystemConfigs.java b/core/src/main/java/org/apache/iceberg/SystemConfigs.java index 9cb345b44480..6dd447175244 100644 --- a/core/src/main/java/org/apache/iceberg/SystemConfigs.java +++ b/core/src/main/java/org/apache/iceberg/SystemConfigs.java @@ -72,7 +72,9 @@ private SystemConfigs() {} 8, Integer::parseUnsignedInt); - /** @deprecated will be removed in 2.0.0; use name mapping instead */ + /** + * @deprecated will be removed in 2.0.0; use name mapping instead + */ @Deprecated public static final ConfigEntry NETFLIX_UNSAFE_PARQUET_ID_FALLBACK_ENABLED = new ConfigEntry<>( diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index abb2ee6cc3e9..d20dd59d2b97 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -966,6 +966,15 @@ private Builder(TableMetadata base) { public Builder withMetadataLocation(String newMetadataLocation) { this.metadataLocation = newMetadataLocation; + if (null != base) { + // carry over lastUpdatedMillis from base and set previousFileLocation to null to avoid + // writing a new metadata log entry + // this is safe since setting metadata location doesn't cause any changes and no other + // changes can be added when metadata location is configured + this.lastUpdatedMillis = base.lastUpdatedMillis(); + this.previousFileLocation = null; + } + return this; } @@ -1494,6 +1503,8 @@ private int addSchemaInternal(Schema schema, int newLastColumnId) { newLastColumnId, lastColumnId); + Schema.checkCompatibility(schema, formatVersion); + int newSchemaId = reuseOrCreateNewSchemaId(schema); boolean schemaFound = schemasById.containsKey(newSchemaId); if (schemaFound && newLastColumnId == lastColumnId) { diff --git a/core/src/main/java/org/apache/iceberg/TableMetadataParser.java b/core/src/main/java/org/apache/iceberg/TableMetadataParser.java index 8bda184142cd..5abfb74974f7 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadataParser.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadataParser.java @@ -122,9 +122,9 @@ public static void write(TableMetadata metadata, OutputFile outputFile) { public static void internalWrite( TableMetadata metadata, OutputFile outputFile, boolean overwrite) { boolean isGzip = Codec.fromFileName(outputFile.location()) == Codec.GZIP; - OutputStream stream = overwrite ? outputFile.createOrOverwrite() : outputFile.create(); - try (OutputStream ou = isGzip ? new GZIPOutputStream(stream) : stream; - OutputStreamWriter writer = new OutputStreamWriter(ou, StandardCharsets.UTF_8)) { + try (OutputStream os = overwrite ? outputFile.createOrOverwrite() : outputFile.create(); + OutputStream gos = isGzip ? new GZIPOutputStream(os) : os; + OutputStreamWriter writer = new OutputStreamWriter(gos, StandardCharsets.UTF_8)) { JsonGenerator generator = JsonUtil.factory().createGenerator(writer); generator.useDefaultPrettyPrinter(); toJson(metadata, generator); @@ -275,9 +275,9 @@ public static TableMetadata read(FileIO io, String path) { public static TableMetadata read(FileIO io, InputFile file) { Codec codec = Codec.fromFileName(file.location()); - try (InputStream is = - codec == Codec.GZIP ? new GZIPInputStream(file.newStream()) : file.newStream()) { - return fromJson(file, JsonUtil.mapper().readValue(is, JsonNode.class)); + try (InputStream is = file.newStream(); + InputStream gis = codec == Codec.GZIP ? new GZIPInputStream(is) : is) { + return fromJson(file, JsonUtil.mapper().readValue(gis, JsonNode.class)); } catch (IOException e) { throw new RuntimeIOException(e, "Failed to read file: %s", file); } diff --git a/core/src/main/java/org/apache/iceberg/TableProperties.java b/core/src/main/java/org/apache/iceberg/TableProperties.java index e41230081615..c137bcd3a2c3 100644 --- a/core/src/main/java/org/apache/iceberg/TableProperties.java +++ b/core/src/main/java/org/apache/iceberg/TableProperties.java @@ -244,12 +244,21 @@ private TableProperties() {} public static final String OBJECT_STORE_ENABLED = "write.object-storage.enabled"; public static final boolean OBJECT_STORE_ENABLED_DEFAULT = false; - /** @deprecated Use {@link #WRITE_DATA_LOCATION} instead. */ + // Excludes the partition values in the path when set to true and object store is enabled + public static final String WRITE_OBJECT_STORE_PARTITIONED_PATHS = + "write.object-storage.partitioned-paths"; + public static final boolean WRITE_OBJECT_STORE_PARTITIONED_PATHS_DEFAULT = true; + + /** + * @deprecated Use {@link #WRITE_DATA_LOCATION} instead. + */ @Deprecated public static final String OBJECT_STORE_PATH = "write.object-storage.path"; public static final String WRITE_LOCATION_PROVIDER_IMPL = "write.location-provider.impl"; - /** @deprecated Use {@link #WRITE_DATA_LOCATION} instead. */ + /** + * @deprecated Use {@link #WRITE_DATA_LOCATION} instead. + */ @Deprecated public static final String WRITE_FOLDER_STORAGE_LOCATION = "write.folder-storage.path"; @@ -266,10 +275,14 @@ private TableProperties() {} public static final String WRITE_PARTITION_SUMMARY_LIMIT = "write.summary.partition-limit"; public static final int WRITE_PARTITION_SUMMARY_LIMIT_DEFAULT = 0; - /** @deprecated will be removed in 2.0.0, writing manifest lists is always enabled */ + /** + * @deprecated will be removed in 2.0.0, writing manifest lists is always enabled + */ @Deprecated public static final String MANIFEST_LISTS_ENABLED = "write.manifest-lists.enabled"; - /** @deprecated will be removed in 2.0.0, writing manifest lists is always enabled */ + /** + * @deprecated will be removed in 2.0.0, writing manifest lists is always enabled + */ @Deprecated public static final boolean MANIFEST_LISTS_ENABLED_DEFAULT = true; public static final String METADATA_COMPRESSION = "write.metadata.compression-codec"; diff --git a/core/src/main/java/org/apache/iceberg/UpdateRequirements.java b/core/src/main/java/org/apache/iceberg/UpdateRequirements.java index 6a5d07d7813d..d92c1a3742fe 100644 --- a/core/src/main/java/org/apache/iceberg/UpdateRequirements.java +++ b/core/src/main/java/org/apache/iceberg/UpdateRequirements.java @@ -124,7 +124,7 @@ private void update(MetadataUpdate.SetSnapshotRef setRef) { } } - private void update(MetadataUpdate.AddSchema update) { + private void update(MetadataUpdate.AddSchema unused) { if (!addedSchema) { if (base != null) { require(new UpdateRequirement.AssertLastAssignedFieldId(base.lastColumnId())); @@ -133,7 +133,7 @@ private void update(MetadataUpdate.AddSchema update) { } } - private void update(MetadataUpdate.SetCurrentSchema update) { + private void update(MetadataUpdate.SetCurrentSchema unused) { if (!setSchemaId) { if (base != null && !isReplace) { // require that the current schema has not changed @@ -143,7 +143,7 @@ private void update(MetadataUpdate.SetCurrentSchema update) { } } - private void update(MetadataUpdate.AddPartitionSpec update) { + private void update(MetadataUpdate.AddPartitionSpec unused) { if (!addedSpec) { if (base != null) { require( @@ -153,7 +153,7 @@ private void update(MetadataUpdate.AddPartitionSpec update) { } } - private void update(MetadataUpdate.SetDefaultPartitionSpec update) { + private void update(MetadataUpdate.SetDefaultPartitionSpec unused) { if (!setSpecId) { if (base != null && !isReplace) { // require that the default spec has not changed @@ -163,7 +163,7 @@ private void update(MetadataUpdate.SetDefaultPartitionSpec update) { } } - private void update(MetadataUpdate.SetDefaultSortOrder update) { + private void update(MetadataUpdate.SetDefaultSortOrder unused) { if (!setOrderId) { if (base != null && !isReplace) { // require that the default write order has not changed diff --git a/core/src/main/java/org/apache/iceberg/V1Metadata.java b/core/src/main/java/org/apache/iceberg/V1Metadata.java index 30b04cd73124..78fa81cebe52 100644 --- a/core/src/main/java/org/apache/iceberg/V1Metadata.java +++ b/core/src/main/java/org/apache/iceberg/V1Metadata.java @@ -399,6 +399,11 @@ public Long pos() { return null; } + @Override + public String manifestLocation() { + return null; + } + @Override public int specId() { return wrapped.specId(); diff --git a/core/src/main/java/org/apache/iceberg/V2Metadata.java b/core/src/main/java/org/apache/iceberg/V2Metadata.java index 8f3b71d39925..be4c3734e40b 100644 --- a/core/src/main/java/org/apache/iceberg/V2Metadata.java +++ b/core/src/main/java/org/apache/iceberg/V2Metadata.java @@ -462,6 +462,11 @@ public Long pos() { return null; } + @Override + public String manifestLocation() { + return null; + } + @Override public int specId() { return wrapped.specId(); diff --git a/core/src/main/java/org/apache/iceberg/V3Metadata.java b/core/src/main/java/org/apache/iceberg/V3Metadata.java index 94e20ea99858..f295af3e109d 100644 --- a/core/src/main/java/org/apache/iceberg/V3Metadata.java +++ b/core/src/main/java/org/apache/iceberg/V3Metadata.java @@ -457,6 +457,11 @@ public void put(int i, Object v) { throw new UnsupportedOperationException("Cannot modify IndexedDataFile wrapper via put"); } + @Override + public String manifestLocation() { + return null; + } + @Override public Long pos() { return null; diff --git a/core/src/main/java/org/apache/iceberg/actions/BaseComputeTableStats.java b/core/src/main/java/org/apache/iceberg/actions/BaseComputeTableStats.java new file mode 100644 index 000000000000..71941af1d7f0 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/actions/BaseComputeTableStats.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.actions; + +import javax.annotation.Nullable; +import org.apache.iceberg.StatisticsFile; +import org.immutables.value.Value; + +@Value.Enclosing +@SuppressWarnings("ImmutablesStyle") +@Value.Style( + typeImmutableEnclosing = "ImmutableComputeTableStats", + visibilityString = "PUBLIC", + builderVisibilityString = "PUBLIC") +interface BaseComputeTableStats extends ComputeTableStats { + + @Value.Immutable + interface Result extends ComputeTableStats.Result { + @Override + @Nullable + StatisticsFile statisticsFile(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/actions/BaseRemoveDanglingDeleteFiles.java b/core/src/main/java/org/apache/iceberg/actions/BaseRemoveDanglingDeleteFiles.java new file mode 100644 index 000000000000..3b5ce9e79a43 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/actions/BaseRemoveDanglingDeleteFiles.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.actions; + +import org.immutables.value.Value; + +@Value.Enclosing +@SuppressWarnings("ImmutablesStyle") +@Value.Style( + typeImmutableEnclosing = "ImmutableRemoveDanglingDeleteFiles", + visibilityString = "PUBLIC", + builderVisibilityString = "PUBLIC") +interface BaseRemoveDanglingDeleteFiles extends RemoveDanglingDeleteFiles { + + @Value.Immutable + interface Result extends RemoveDanglingDeleteFiles.Result {} +} diff --git a/core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFiles.java b/core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFiles.java index 953439484a15..2faa1f1b756c 100644 --- a/core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFiles.java +++ b/core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFiles.java @@ -55,6 +55,12 @@ default long rewrittenBytesCount() { return RewriteDataFiles.Result.super.rewrittenBytesCount(); } + @Override + @Value.Default + default int removedDeleteFilesCount() { + return RewriteDataFiles.Result.super.removedDeleteFilesCount(); + } + @Override @Value.Default default int failedDataFilesCount() { diff --git a/core/src/main/java/org/apache/iceberg/actions/BaseRewriteTablePath.java b/core/src/main/java/org/apache/iceberg/actions/BaseRewriteTablePath.java new file mode 100644 index 000000000000..1fb343da92fe --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/actions/BaseRewriteTablePath.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.actions; + +import org.immutables.value.Value; + +@Value.Enclosing +@SuppressWarnings("ImmutablesStyle") +@Value.Style( + typeImmutableEnclosing = "ImmutableRewriteTablePath", + visibilityString = "PUBLIC", + builderVisibilityString = "PUBLIC") +interface BaseRewriteTablePath extends RewriteTablePath { + + @Value.Immutable + interface Result extends RewriteTablePath.Result {} +} diff --git a/core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitManager.java b/core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitManager.java index 45b4bcf0a4d9..03d23231c0f1 100644 --- a/core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitManager.java +++ b/core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitManager.java @@ -20,14 +20,13 @@ import java.util.Map; import java.util.Set; -import org.apache.iceberg.DataFile; import org.apache.iceberg.RewriteFiles; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.CleanableFailure; import org.apache.iceberg.exceptions.CommitStateUnknownException; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.DataFileSet; import org.apache.iceberg.util.Tasks; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -73,8 +72,8 @@ public RewriteDataFilesCommitManager( * @param fileGroups fileSets to commit */ public void commitFileGroups(Set fileGroups) { - Set rewrittenDataFiles = Sets.newHashSet(); - Set addedDataFiles = Sets.newHashSet(); + DataFileSet rewrittenDataFiles = DataFileSet.create(); + DataFileSet addedDataFiles = DataFileSet.create(); for (RewriteFileGroup group : fileGroups) { rewrittenDataFiles.addAll(group.rewrittenFiles()); addedDataFiles.addAll(group.addedFiles()); diff --git a/core/src/main/java/org/apache/iceberg/actions/RewriteFileGroup.java b/core/src/main/java/org/apache/iceberg/actions/RewriteFileGroup.java index f816b5d7a4f6..dfc9842780f5 100644 --- a/core/src/main/java/org/apache/iceberg/actions/RewriteFileGroup.java +++ b/core/src/main/java/org/apache/iceberg/actions/RewriteFileGroup.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.actions; -import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Set; @@ -29,6 +28,7 @@ import org.apache.iceberg.actions.RewriteDataFiles.FileGroupInfo; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.DataFileSet; /** * Container class representing a set of files to be rewritten by a RewriteAction and the new files @@ -38,7 +38,7 @@ public class RewriteFileGroup { private final FileGroupInfo info; private final List fileScanTasks; - private Set addedFiles = Collections.emptySet(); + private DataFileSet addedFiles = DataFileSet.create(); public RewriteFileGroup(FileGroupInfo info, List fileScanTasks) { this.info = info; @@ -54,11 +54,13 @@ public List fileScans() { } public void setOutputFiles(Set files) { - addedFiles = files; + addedFiles = DataFileSet.of(files); } public Set rewrittenFiles() { - return fileScans().stream().map(FileScanTask::file).collect(Collectors.toSet()); + return fileScans().stream() + .map(FileScanTask::file) + .collect(Collectors.toCollection(DataFileSet::create)); } public Set addedFiles() { @@ -106,7 +108,7 @@ public static Comparator comparator(RewriteJobOrder rewriteJob case FILES_DESC: return Comparator.comparing(RewriteFileGroup::numFiles, Comparator.reverseOrder()); default: - return (fileGroupOne, fileGroupTwo) -> 0; + return (unused, unused2) -> 0; } } } diff --git a/core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesGroup.java b/core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesGroup.java index b8ebe528deea..d1c688417a64 100644 --- a/core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesGroup.java +++ b/core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesGroup.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.actions; -import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Set; @@ -30,6 +29,7 @@ import org.apache.iceberg.actions.RewritePositionDeleteFiles.FileGroupRewriteResult; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.DeleteFileSet; /** * Container class representing a set of position delete files to be rewritten by a {@link @@ -40,7 +40,7 @@ public class RewritePositionDeletesGroup { private final List tasks; private final long maxRewrittenDataSequenceNumber; - private Set addedDeleteFiles = Collections.emptySet(); + private DeleteFileSet addedDeleteFiles = DeleteFileSet.create(); public RewritePositionDeletesGroup(FileGroupInfo info, List tasks) { Preconditions.checkArgument(!tasks.isEmpty(), "Tasks must not be empty"); @@ -59,7 +59,7 @@ public List tasks() { } public void setOutputFiles(Set files) { - addedDeleteFiles = files; + addedDeleteFiles = DeleteFileSet.of(files); } public long maxRewrittenDataSequenceNumber() { @@ -67,7 +67,9 @@ public long maxRewrittenDataSequenceNumber() { } public Set rewrittenDeleteFiles() { - return tasks().stream().map(PositionDeletesScanTask::file).collect(Collectors.toSet()); + return tasks().stream() + .map(PositionDeletesScanTask::file) + .collect(Collectors.toCollection(DeleteFileSet::create)); } public Set addedDeleteFiles() { @@ -127,7 +129,7 @@ public static Comparator comparator(RewriteJobOrder return Comparator.comparing( RewritePositionDeletesGroup::numRewrittenDeleteFiles, Comparator.reverseOrder()); default: - return (fileGroupOne, fileGroupTwo) -> 0; + return (unused, unused2) -> 0; } } } diff --git a/core/src/main/java/org/apache/iceberg/avro/AvroWithPartnerVisitor.java b/core/src/main/java/org/apache/iceberg/avro/AvroWithPartnerVisitor.java index b23b195d959a..692c1ead3fbf 100644 --- a/core/src/main/java/org/apache/iceberg/avro/AvroWithPartnerVisitor.java +++ b/core/src/main/java/org/apache/iceberg/avro/AvroWithPartnerVisitor.java @@ -37,7 +37,7 @@ public interface PartnerAccessors

{ P listElementPartner(P partnerList); } - static class FieldIDAccessors implements AvroWithPartnerVisitor.PartnerAccessors { + public static class FieldIDAccessors implements AvroWithPartnerVisitor.PartnerAccessors { private static final FieldIDAccessors INSTANCE = new FieldIDAccessors(); public static FieldIDAccessors get() { diff --git a/core/src/main/java/org/apache/iceberg/avro/GenericAvroReader.java b/core/src/main/java/org/apache/iceberg/avro/GenericAvroReader.java index 93bfa2398466..bfdb65acf1c2 100644 --- a/core/src/main/java/org/apache/iceberg/avro/GenericAvroReader.java +++ b/core/src/main/java/org/apache/iceberg/avro/GenericAvroReader.java @@ -28,11 +28,8 @@ import org.apache.avro.generic.IndexedRecord; import org.apache.avro.io.DatumReader; import org.apache.avro.io.Decoder; -import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.common.DynClasses; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.Pair; @@ -43,20 +40,20 @@ public class GenericAvroReader private final Types.StructType expectedType; private ClassLoader loader = Thread.currentThread().getContextClassLoader(); private Map renames = ImmutableMap.of(); - private final Map idToConstant = ImmutableMap.of(); + private final Map idToConstant = ImmutableMap.of(); private Schema fileSchema = null; private ValueReader reader = null; - public static GenericAvroReader create(org.apache.iceberg.Schema schema) { - return new GenericAvroReader<>(schema); + public static GenericAvroReader create(org.apache.iceberg.Schema expectedSchema) { + return new GenericAvroReader<>(expectedSchema); } - public static GenericAvroReader create(Schema schema) { - return new GenericAvroReader<>(schema); + public static GenericAvroReader create(Schema readSchema) { + return new GenericAvroReader<>(readSchema); } - GenericAvroReader(org.apache.iceberg.Schema readSchema) { - this.expectedType = readSchema.asStruct(); + GenericAvroReader(org.apache.iceberg.Schema expectedSchema) { + this.expectedType = expectedSchema.asStruct(); } GenericAvroReader(Schema readSchema) { @@ -111,46 +108,13 @@ private ResolvingReadBuilder(Types.StructType expectedType, String rootName) { @Override public ValueReader record(Type partner, Schema record, List> fieldResults) { - Types.StructType expected = partner != null ? partner.asStructType() : null; - Map idToPos = idToPos(expected); - - List>> readPlan = Lists.newArrayList(); - List fileFields = record.getFields(); - for (int pos = 0; pos < fileFields.size(); pos += 1) { - Schema.Field field = fileFields.get(pos); - ValueReader fieldReader = fieldResults.get(pos); - Integer fieldId = AvroSchemaUtil.fieldId(field); - Integer projectionPos = idToPos.remove(fieldId); - - Object constant = idToConstant.get(fieldId); - if (projectionPos != null && constant != null) { - readPlan.add( - Pair.of(projectionPos, ValueReaders.replaceWithConstant(fieldReader, constant))); - } else { - readPlan.add(Pair.of(projectionPos, fieldReader)); - } + if (partner == null) { + return ValueReaders.skipStruct(fieldResults); } - // handle any expected columns that are not in the data file - for (Map.Entry idAndPos : idToPos.entrySet()) { - int fieldId = idAndPos.getKey(); - int pos = idAndPos.getValue(); - - Object constant = idToConstant.get(fieldId); - Types.NestedField field = expected.field(fieldId); - if (constant != null) { - readPlan.add(Pair.of(pos, ValueReaders.constant(constant))); - } else if (fieldId == MetadataColumns.IS_DELETED.fieldId()) { - readPlan.add(Pair.of(pos, ValueReaders.constant(false))); - } else if (fieldId == MetadataColumns.ROW_POSITION.fieldId()) { - readPlan.add(Pair.of(pos, ValueReaders.positions())); - } else if (field.isOptional()) { - readPlan.add(Pair.of(pos, ValueReaders.constant(null))); - } else { - throw new IllegalArgumentException( - String.format("Missing required field: %s", field.name())); - } - } + Types.StructType expected = partner.asStructType(); + List>> readPlan = + ValueReaders.buildReadPlan(expected, record, fieldResults, idToConstant); return recordReader(readPlan, avroSchemas.get(partner), record.getFullName()); } @@ -264,19 +228,5 @@ public ValueReader primitive(Type partner, Schema primitive) { throw new IllegalArgumentException("Unsupported type: " + primitive); } } - - private Map idToPos(Types.StructType struct) { - Map idToPos = Maps.newHashMap(); - - if (struct != null) { - List fields = struct.fields(); - for (int pos = 0; pos < fields.size(); pos += 1) { - Types.NestedField field = fields.get(pos); - idToPos.put(field.fieldId(), pos); - } - } - - return idToPos; - } } } diff --git a/core/src/main/java/org/apache/iceberg/avro/InternalReader.java b/core/src/main/java/org/apache/iceberg/avro/InternalReader.java new file mode 100644 index 000000000000..ca83ce2ba7cd --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/avro/InternalReader.java @@ -0,0 +1,252 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.avro; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.function.Supplier; +import org.apache.avro.LogicalType; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.io.DatumReader; +import org.apache.avro.io.Decoder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; + +/** + * A reader that produces Iceberg's internal in-memory object model. + * + *

Iceberg's internal in-memory object model produces the types defined in {@link + * Type.TypeID#javaClass()}. + * + * @param Java type returned by the reader + */ +public class InternalReader implements DatumReader, SupportsRowPosition { + private static final int ROOT_ID = -1; + + private final Types.StructType expectedType; + private final Map> typeMap = Maps.newHashMap(); + private final Map idToConstant = ImmutableMap.of(); + private Schema fileSchema = null; + private ValueReader reader = null; + + public static InternalReader create(org.apache.iceberg.Schema schema) { + return new InternalReader<>(schema); + } + + InternalReader(org.apache.iceberg.Schema readSchema) { + this.expectedType = readSchema.asStruct(); + } + + @SuppressWarnings("unchecked") + private void initReader() { + this.reader = + (ValueReader) + AvroWithPartnerVisitor.visit( + Pair.of(ROOT_ID, expectedType), + fileSchema, + new ResolvingReadBuilder(), + AccessByID.instance()); + } + + @Override + public void setSchema(Schema schema) { + this.fileSchema = schema; + initReader(); + } + + public InternalReader setRootType(Class rootClass) { + typeMap.put(ROOT_ID, rootClass); + return this; + } + + public InternalReader setCustomType(int fieldId, Class structClass) { + typeMap.put(fieldId, structClass); + return this; + } + + @Override + public void setRowPositionSupplier(Supplier posSupplier) { + if (reader instanceof SupportsRowPosition) { + ((SupportsRowPosition) reader).setRowPositionSupplier(posSupplier); + } + } + + @Override + public T read(T reuse, Decoder decoder) throws IOException { + return reader.read(decoder, reuse); + } + + private class ResolvingReadBuilder + extends AvroWithPartnerVisitor, ValueReader> { + @Override + public ValueReader record( + Pair partner, Schema record, List> fieldResults) { + if (partner == null) { + return ValueReaders.skipStruct(fieldResults); + } + + Types.StructType expected = partner.second().asStructType(); + List>> readPlan = + ValueReaders.buildReadPlan(expected, record, fieldResults, idToConstant); + + return structReader(readPlan, partner.first(), expected); + } + + private ValueReader structReader( + List>> readPlan, int fieldId, Types.StructType struct) { + + Class structClass = typeMap.get(fieldId); + if (structClass != null) { + return InternalReaders.struct(struct, structClass, readPlan); + } else { + return InternalReaders.struct(struct, readPlan); + } + } + + @Override + public ValueReader union( + Pair partner, Schema union, List> options) { + return ValueReaders.union(options); + } + + @Override + public ValueReader arrayMap( + Pair partner, + Schema map, + ValueReader keyReader, + ValueReader valueReader) { + return ValueReaders.arrayMap(keyReader, valueReader); + } + + @Override + public ValueReader array( + Pair partner, Schema array, ValueReader elementReader) { + return ValueReaders.array(elementReader); + } + + @Override + public ValueReader map(Pair partner, Schema map, ValueReader valueReader) { + return ValueReaders.map(ValueReaders.strings(), valueReader); + } + + @Override + public ValueReader primitive(Pair partner, Schema primitive) { + LogicalType logicalType = primitive.getLogicalType(); + if (logicalType != null) { + switch (logicalType.getName()) { + case "date": + return ValueReaders.ints(); + + case "time-micros": + return ValueReaders.longs(); + + case "timestamp-millis": + // adjust to microseconds + ValueReader longs = ValueReaders.longs(); + return (ValueReader) (decoder, ignored) -> longs.read(decoder, null) * 1000L; + + case "timestamp-micros": + return ValueReaders.longs(); + + case "decimal": + return ValueReaders.decimal( + ValueReaders.decimalBytesReader(primitive), + ((LogicalTypes.Decimal) logicalType).getScale()); + + case "uuid": + return ValueReaders.uuids(); + + default: + throw new IllegalArgumentException("Unknown logical type: " + logicalType); + } + } + + switch (primitive.getType()) { + case NULL: + return ValueReaders.nulls(); + case BOOLEAN: + return ValueReaders.booleans(); + case INT: + if (partner != null && partner.second().typeId() == Type.TypeID.LONG) { + return ValueReaders.intsAsLongs(); + } + return ValueReaders.ints(); + case LONG: + return ValueReaders.longs(); + case FLOAT: + if (partner != null && partner.second().typeId() == Type.TypeID.DOUBLE) { + return ValueReaders.floatsAsDoubles(); + } + return ValueReaders.floats(); + case DOUBLE: + return ValueReaders.doubles(); + case STRING: + return ValueReaders.strings(); + case FIXED: + return ValueReaders.fixed(primitive); + case BYTES: + return ValueReaders.byteBuffers(); + case ENUM: + return ValueReaders.enums(primitive.getEnumSymbols()); + default: + throw new IllegalArgumentException("Unsupported type: " + primitive); + } + } + } + + private static class AccessByID + implements AvroWithPartnerVisitor.PartnerAccessors> { + private static final AccessByID INSTANCE = new AccessByID(); + + public static AccessByID instance() { + return INSTANCE; + } + + @Override + public Pair fieldPartner( + Pair partner, Integer fieldId, String name) { + Types.NestedField field = partner.second().asStructType().field(fieldId); + return field != null ? Pair.of(field.fieldId(), field.type()) : null; + } + + @Override + public Pair mapKeyPartner(Pair partner) { + Types.MapType map = partner.second().asMapType(); + return Pair.of(map.keyId(), map.keyType()); + } + + @Override + public Pair mapValuePartner(Pair partner) { + Types.MapType map = partner.second().asMapType(); + return Pair.of(map.valueId(), map.valueType()); + } + + @Override + public Pair listElementPartner(Pair partner) { + Types.ListType list = partner.second().asListType(); + return Pair.of(list.elementId(), list.elementType()); + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/avro/InternalReaders.java b/core/src/main/java/org/apache/iceberg/avro/InternalReaders.java new file mode 100644 index 000000000000..6136bae052ae --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/avro/InternalReaders.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.avro; + +import java.util.List; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.common.DynConstructors; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; + +class InternalReaders { + private InternalReaders() {} + + static ValueReader struct( + Types.StructType struct, List>> readPlan) { + return new RecordReader(readPlan, struct); + } + + static ValueReader struct( + Types.StructType struct, Class structClass, List>> readPlan) { + return new PlannedStructLikeReader<>(readPlan, struct, structClass); + } + + private static class PlannedStructLikeReader + extends ValueReaders.PlannedStructReader { + private final Types.StructType structType; + private final Class structClass; + private final DynConstructors.Ctor ctor; + + private PlannedStructLikeReader( + List>> readPlan, + Types.StructType structType, + Class structClass) { + super(readPlan); + this.structType = structType; + this.structClass = structClass; + this.ctor = + DynConstructors.builder(StructLike.class) + .hiddenImpl(structClass, Types.StructType.class) + .hiddenImpl(structClass) + .build(); + } + + @Override + protected S reuseOrCreate(Object reuse) { + if (structClass.isInstance(reuse)) { + return structClass.cast(reuse); + } else { + return ctor.newInstance(structType); + } + } + + @Override + protected Object get(S struct, int pos) { + return struct.get(pos, Object.class); + } + + @Override + protected void set(S struct, int pos, Object value) { + struct.set(pos, value); + } + } + + private static class RecordReader extends ValueReaders.PlannedStructReader { + private final Types.StructType structType; + + private RecordReader( + List>> readPlan, Types.StructType structType) { + super(readPlan); + this.structType = structType; + } + + @Override + protected GenericRecord reuseOrCreate(Object reuse) { + if (reuse instanceof GenericRecord) { + return (GenericRecord) reuse; + } else { + return GenericRecord.create(structType); + } + } + + @Override + protected Object get(GenericRecord struct, int pos) { + return struct.get(pos); + } + + @Override + protected void set(GenericRecord struct, int pos, Object value) { + struct.set(pos, value); + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/avro/SupportsIndexProjection.java b/core/src/main/java/org/apache/iceberg/avro/SupportsIndexProjection.java new file mode 100644 index 000000000000..fa4ffa5aec13 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/avro/SupportsIndexProjection.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.avro; + +import java.io.Serializable; +import java.util.List; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.types.Types; + +public abstract class SupportsIndexProjection implements StructLike, Serializable { + private final int[] fromProjectionPos; + + /** Noop constructor that does not project fields */ + protected SupportsIndexProjection(int size) { + this.fromProjectionPos = new int[size]; + for (int i = 0; i < fromProjectionPos.length; i++) { + fromProjectionPos[i] = i; + } + } + + /** Base constructor for building the type mapping */ + protected SupportsIndexProjection(Types.StructType baseType, Types.StructType projectionType) { + List allFields = baseType.fields(); + List fields = projectionType.fields(); + + this.fromProjectionPos = new int[fields.size()]; + for (int i = 0; i < fromProjectionPos.length; i += 1) { + boolean found = false; + for (int j = 0; j < allFields.size(); j += 1) { + if (fields.get(i).fieldId() == allFields.get(j).fieldId()) { + found = true; + fromProjectionPos[i] = j; + } + } + + if (!found) { + throw new IllegalArgumentException("Cannot find projected field: " + fields.get(i)); + } + } + } + + /** Copy constructor */ + protected SupportsIndexProjection(SupportsIndexProjection toCopy) { + this.fromProjectionPos = toCopy.fromProjectionPos; + } + + protected abstract T internalGet(int pos, Class javaClass); + + protected abstract void internalSet(int pos, T value); + + private int pos(int basePos) { + return fromProjectionPos[basePos]; + } + + @Override + public int size() { + return fromProjectionPos.length; + } + + @Override + public T get(int basePos, Class javaClass) { + return internalGet(pos(basePos), javaClass); + } + + @Override + public void set(int basePos, T value) { + internalSet(pos(basePos), value); + } +} diff --git a/core/src/main/java/org/apache/iceberg/avro/ValueReaders.java b/core/src/main/java/org/apache/iceberg/avro/ValueReaders.java index d530bc1854e1..67f53d3636a6 100644 --- a/core/src/main/java/org/apache/iceberg/avro/ValueReaders.java +++ b/core/src/main/java/org/apache/iceberg/avro/ValueReaders.java @@ -181,6 +181,83 @@ public static ValueReader record( return new PlannedIndexedReader<>(recordSchema, recordClass, readPlan); } + public static ValueReader skipStruct(List> readers) { + return new SkipStructReader(readers); + } + + /** + * Builds a read plan for record classes that use planned reads instead of a ResolvingDecoder. + * + * @param expected expected StructType + * @param record Avro record schema + * @param fieldReaders list of readers for each field in the Avro record schema + * @param idToConstant a map of field ID to constants values + * @return a read plan that is a list of (position, reader) pairs + */ + public static List>> buildReadPlan( + Types.StructType expected, + Schema record, + List> fieldReaders, + Map idToConstant) { + Map idToPos = idToPos(expected); + + List>> readPlan = Lists.newArrayList(); + List fileFields = record.getFields(); + for (int pos = 0; pos < fileFields.size(); pos += 1) { + Schema.Field field = fileFields.get(pos); + ValueReader fieldReader = fieldReaders.get(pos); + Integer fieldId = AvroSchemaUtil.fieldId(field); + Integer projectionPos = idToPos.remove(fieldId); + + Object constant = idToConstant.get(fieldId); + if (projectionPos != null && constant != null) { + readPlan.add( + Pair.of(projectionPos, ValueReaders.replaceWithConstant(fieldReader, constant))); + } else { + readPlan.add(Pair.of(projectionPos, fieldReader)); + } + } + + // handle any expected columns that are not in the data file + for (Map.Entry idAndPos : idToPos.entrySet()) { + int fieldId = idAndPos.getKey(); + int pos = idAndPos.getValue(); + + Object constant = idToConstant.get(fieldId); + Types.NestedField field = expected.field(fieldId); + if (constant != null) { + readPlan.add(Pair.of(pos, ValueReaders.constant(constant))); + } else if (field.initialDefault() != null) { + readPlan.add(Pair.of(pos, ValueReaders.constant(field.initialDefault()))); + } else if (fieldId == MetadataColumns.IS_DELETED.fieldId()) { + readPlan.add(Pair.of(pos, ValueReaders.constant(false))); + } else if (fieldId == MetadataColumns.ROW_POSITION.fieldId()) { + readPlan.add(Pair.of(pos, ValueReaders.positions())); + } else if (field.isOptional()) { + readPlan.add(Pair.of(pos, ValueReaders.constant(null))); + } else { + throw new IllegalArgumentException( + String.format("Missing required field: %s", field.name())); + } + } + + return readPlan; + } + + private static Map idToPos(Types.StructType struct) { + Map idToPos = Maps.newHashMap(); + + if (struct != null) { + List fields = struct.fields(); + for (int pos = 0; pos < fields.size(); pos += 1) { + Types.NestedField field = fields.get(pos); + idToPos.put(field.fieldId(), pos); + } + } + + return idToPos; + } + private static class NullReader implements ValueReader { private static final NullReader INSTANCE = new NullReader(); @@ -777,6 +854,27 @@ public void skip(Decoder decoder) throws IOException { } } + private static class SkipStructReader implements ValueReader { + private final ValueReader[] readers; + + private SkipStructReader(List> readers) { + this.readers = readers.toArray(ValueReader[]::new); + } + + @Override + public Void read(Decoder decoder, Object reuse) throws IOException { + skip(decoder); + return null; + } + + @Override + public void skip(Decoder decoder) throws IOException { + for (ValueReader reader : readers) { + reader.skip(decoder); + } + } + } + public abstract static class PlannedStructReader implements ValueReader, SupportsRowPosition { private final ValueReader[] readers; diff --git a/core/src/main/java/org/apache/iceberg/data/avro/IcebergDecoder.java b/core/src/main/java/org/apache/iceberg/data/avro/IcebergDecoder.java index 94fedbcfce91..89513b7e0bed 100644 --- a/core/src/main/java/org/apache/iceberg/data/avro/IcebergDecoder.java +++ b/core/src/main/java/org/apache/iceberg/data/avro/IcebergDecoder.java @@ -23,6 +23,7 @@ import java.io.UncheckedIOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; +import java.util.Locale; import java.util.Map; import org.apache.avro.AvroRuntimeException; import org.apache.avro.Schema; @@ -138,7 +139,8 @@ public D decode(InputStream stream, D reuse) throws IOException { if (IcebergEncoder.V1_HEADER[0] != header[0] || IcebergEncoder.V1_HEADER[1] != header[1]) { throw new BadHeaderException( - String.format("Unrecognized header bytes: 0x%02X 0x%02X", header[0], header[1])); + String.format( + Locale.ROOT, "Unrecognized header bytes: 0x%02X 0x%02X", header[0], header[1])); } RawDecoder decoder = getDecoder(FP_BUFFER.get().getLong(2)); diff --git a/core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java b/core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java index 72f1e00e498a..a1b57a38666d 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java +++ b/core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java @@ -18,17 +18,35 @@ */ package org.apache.iceberg.deletes; +import java.util.Collection; +import java.util.List; +import java.util.function.LongConsumer; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.roaringbitmap.longlong.Roaring64Bitmap; class BitmapPositionDeleteIndex implements PositionDeleteIndex { private final Roaring64Bitmap roaring64Bitmap; + private final List deleteFiles; BitmapPositionDeleteIndex() { - roaring64Bitmap = new Roaring64Bitmap(); + this.roaring64Bitmap = new Roaring64Bitmap(); + this.deleteFiles = Lists.newArrayList(); + } + + BitmapPositionDeleteIndex(Collection deleteFiles) { + this.roaring64Bitmap = new Roaring64Bitmap(); + this.deleteFiles = Lists.newArrayList(deleteFiles); + } + + BitmapPositionDeleteIndex(DeleteFile deleteFile) { + this.roaring64Bitmap = new Roaring64Bitmap(); + this.deleteFiles = deleteFile != null ? Lists.newArrayList(deleteFile) : Lists.newArrayList(); } void merge(BitmapPositionDeleteIndex that) { roaring64Bitmap.or(that.roaring64Bitmap); + deleteFiles.addAll(that.deleteFiles); } @Override @@ -41,6 +59,16 @@ public void delete(long posStart, long posEnd) { roaring64Bitmap.addRange(posStart, posEnd); } + @Override + public void merge(PositionDeleteIndex that) { + if (that instanceof BitmapPositionDeleteIndex) { + merge((BitmapPositionDeleteIndex) that); + } else { + that.forEach(this::delete); + deleteFiles.addAll(that.deleteFiles()); + } + } + @Override public boolean isDeleted(long position) { return roaring64Bitmap.contains(position); @@ -50,4 +78,14 @@ public boolean isDeleted(long position) { public boolean isEmpty() { return roaring64Bitmap.isEmpty(); } + + @Override + public void forEach(LongConsumer consumer) { + roaring64Bitmap.forEach(consumer::accept); + } + + @Override + public Collection deleteFiles() { + return deleteFiles; + } } diff --git a/core/src/main/java/org/apache/iceberg/deletes/Deletes.java b/core/src/main/java/org/apache/iceberg/deletes/Deletes.java index cef57cd16726..0838e9c0dd6e 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/Deletes.java +++ b/core/src/main/java/org/apache/iceberg/deletes/Deletes.java @@ -26,13 +26,11 @@ import java.util.function.Function; import java.util.function.Predicate; import org.apache.iceberg.Accessor; +import org.apache.iceberg.DeleteFile; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Schema; import org.apache.iceberg.StructLike; -import org.apache.iceberg.io.CloseableGroup; import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.CloseableIterator; -import org.apache.iceberg.io.FilterIterator; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -44,13 +42,9 @@ import org.apache.iceberg.util.SortedMerge; import org.apache.iceberg.util.StructLikeSet; import org.apache.iceberg.util.ThreadPools; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class Deletes { - private static final Logger LOG = LoggerFactory.getLogger(Deletes.class); - private static final Schema POSITION_DELETE_SCHEMA = new Schema(MetadataColumns.DELETE_FILE_PATH, MetadataColumns.DELETE_FILE_POS); @@ -130,6 +124,11 @@ public static StructLikeSet toEqualitySet( } } + public static CharSequenceMap toPositionIndexes( + CloseableIterable posDeletes) { + return toPositionIndexes(posDeletes, null /* unknown delete file */); + } + /** * Builds a map of position delete indexes by path. * @@ -138,10 +137,11 @@ public static StructLikeSet toEqualitySet( * entire delete file content is needed (e.g. caching). * * @param posDeletes position deletes + * @param file the source delete file for the deletes * @return the map of position delete indexes by path */ public static CharSequenceMap toPositionIndexes( - CloseableIterable posDeletes) { + CloseableIterable posDeletes, DeleteFile file) { CharSequenceMap indexes = CharSequenceMap.create(); try (CloseableIterable deletes = posDeletes) { @@ -149,7 +149,7 @@ public static CharSequenceMap toPosi CharSequence filePath = (CharSequence) FILENAME_ACCESSOR.get(delete); long position = (long) POSITION_ACCESSOR.get(delete); PositionDeleteIndex index = - indexes.computeIfAbsent(filePath, key -> new BitmapPositionDeleteIndex()); + indexes.computeIfAbsent(filePath, key -> new BitmapPositionDeleteIndex(file)); index.delete(position); } } catch (IOException e) { @@ -159,11 +159,33 @@ public static CharSequenceMap toPosi return indexes; } + public static PositionDeleteIndex toPositionIndex( + CharSequence dataLocation, CloseableIterable posDeletes, DeleteFile file) { + CloseableIterable positions = extractPositions(dataLocation, posDeletes); + List files = ImmutableList.of(file); + return toPositionIndex(positions, files); + } + + private static CloseableIterable extractPositions( + CharSequence dataLocation, CloseableIterable rows) { + DataFileFilter filter = new DataFileFilter<>(dataLocation); + CloseableIterable filteredRows = filter.filter(rows); + return CloseableIterable.transform(filteredRows, row -> (Long) POSITION_ACCESSOR.get(row)); + } + + /** + * @deprecated since 1.7.0, will be removed in 1.8.0; use delete loaders. + */ + @Deprecated public static PositionDeleteIndex toPositionIndex( CharSequence dataLocation, List> deleteFiles) { return toPositionIndex(dataLocation, deleteFiles, ThreadPools.getDeleteWorkerPool()); } + /** + * @deprecated since 1.7.0, will be removed in 1.8.0; use delete loaders. + */ + @Deprecated public static PositionDeleteIndex toPositionIndex( CharSequence dataLocation, List> deleteFiles, @@ -183,8 +205,13 @@ public static PositionDeleteIndex toPositionIndex( } public static PositionDeleteIndex toPositionIndex(CloseableIterable posDeletes) { + return toPositionIndex(posDeletes, ImmutableList.of()); + } + + private static PositionDeleteIndex toPositionIndex( + CloseableIterable posDeletes, List files) { try (CloseableIterable deletes = posDeletes) { - PositionDeleteIndex positionDeleteIndex = new BitmapPositionDeleteIndex(); + PositionDeleteIndex positionDeleteIndex = new BitmapPositionDeleteIndex(files); deletes.forEach(positionDeleteIndex::delete); return positionDeleteIndex; } catch (IOException e) { @@ -192,6 +219,10 @@ public static PositionDeleteIndex toPositionIndex(CloseableIterable posDel } } + /** + * @deprecated since 1.7.0, will be removed in 1.8.0. + */ + @Deprecated public static CloseableIterable streamingFilter( CloseableIterable rows, Function rowToPosition, @@ -199,20 +230,32 @@ public static CloseableIterable streamingFilter( return streamingFilter(rows, rowToPosition, posDeletes, new DeleteCounter()); } + /** + * @deprecated since 1.7.0, will be removed in 1.8.0. + */ + @Deprecated public static CloseableIterable streamingFilter( CloseableIterable rows, Function rowToPosition, CloseableIterable posDeletes, DeleteCounter counter) { - return new PositionStreamDeleteFilter<>(rows, rowToPosition, posDeletes, counter); + PositionDeleteIndex positionIndex = toPositionIndex(posDeletes); + Predicate isDeleted = row -> positionIndex.isDeleted(rowToPosition.apply(row)); + return filterDeleted(rows, isDeleted, counter); } + /** + * @deprecated since 1.7.0, will be removed in 1.8.0. + */ + @Deprecated public static CloseableIterable streamingMarker( CloseableIterable rows, Function rowToPosition, CloseableIterable posDeletes, - Consumer markDeleted) { - return new PositionStreamDeleteMarker<>(rows, rowToPosition, posDeletes, markDeleted); + Consumer markRowDeleted) { + PositionDeleteIndex positionIndex = toPositionIndex(posDeletes); + Predicate isDeleted = row -> positionIndex.isDeleted(rowToPosition.apply(row)); + return markDeleted(rows, isDeleted, markRowDeleted); } public static CloseableIterable deletePositions( @@ -248,148 +291,6 @@ protected boolean shouldKeep(T row) { } } - private abstract static class PositionStreamDeleteIterable extends CloseableGroup - implements CloseableIterable { - private final CloseableIterable rows; - private final CloseableIterator deletePosIterator; - private final Function rowToPosition; - private long nextDeletePos; - - PositionStreamDeleteIterable( - CloseableIterable rows, - Function rowToPosition, - CloseableIterable deletePositions) { - this.rows = rows; - this.rowToPosition = rowToPosition; - this.deletePosIterator = deletePositions.iterator(); - } - - @Override - public CloseableIterator iterator() { - CloseableIterator iter; - if (deletePosIterator.hasNext()) { - nextDeletePos = deletePosIterator.next(); - iter = applyDelete(rows.iterator(), deletePosIterator); - } else { - iter = rows.iterator(); - } - - addCloseable(iter); - addCloseable(deletePosIterator); - - return iter; - } - - boolean isDeleted(T row) { - long currentPos = rowToPosition.apply(row); - if (currentPos < nextDeletePos) { - return false; - } - - // consume delete positions until the next is past the current position - boolean isDeleted = currentPos == nextDeletePos; - while (deletePosIterator.hasNext() && nextDeletePos <= currentPos) { - this.nextDeletePos = deletePosIterator.next(); - if (!isDeleted && currentPos == nextDeletePos) { - // if any delete position matches the current position - isDeleted = true; - } - } - - return isDeleted; - } - - protected abstract CloseableIterator applyDelete( - CloseableIterator items, CloseableIterator deletePositions); - } - - private static class PositionStreamDeleteFilter extends PositionStreamDeleteIterable { - private final DeleteCounter counter; - - PositionStreamDeleteFilter( - CloseableIterable rows, - Function rowToPosition, - CloseableIterable deletePositions, - DeleteCounter counter) { - super(rows, rowToPosition, deletePositions); - this.counter = counter; - } - - @Override - protected CloseableIterator applyDelete( - CloseableIterator items, CloseableIterator deletePositions) { - return new FilterIterator(items) { - @Override - protected boolean shouldKeep(T item) { - boolean deleted = isDeleted(item); - if (deleted) { - counter.increment(); - } - - return !deleted; - } - - @Override - public void close() { - try { - deletePositions.close(); - } catch (IOException e) { - LOG.warn("Error closing delete file", e); - } - super.close(); - } - }; - } - } - - private static class PositionStreamDeleteMarker extends PositionStreamDeleteIterable { - private final Consumer markDeleted; - - PositionStreamDeleteMarker( - CloseableIterable rows, - Function rowToPosition, - CloseableIterable deletePositions, - Consumer markDeleted) { - super(rows, rowToPosition, deletePositions); - this.markDeleted = markDeleted; - } - - @Override - protected CloseableIterator applyDelete( - CloseableIterator items, CloseableIterator deletePositions) { - - return new CloseableIterator() { - @Override - public void close() { - try { - deletePositions.close(); - } catch (IOException e) { - LOG.warn("Error closing delete file", e); - } - try { - items.close(); - } catch (IOException e) { - LOG.warn("Error closing data file", e); - } - } - - @Override - public boolean hasNext() { - return items.hasNext(); - } - - @Override - public T next() { - T row = items.next(); - if (isDeleted(row)) { - markDeleted.accept(row); - } - return row; - } - }; - } - } - private static class DataFileFilter extends Filter { private final CharSequence dataLocation; diff --git a/core/src/main/java/org/apache/iceberg/deletes/PositionDelete.java b/core/src/main/java/org/apache/iceberg/deletes/PositionDelete.java index 655428ce7713..57e188567f68 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/PositionDelete.java +++ b/core/src/main/java/org/apache/iceberg/deletes/PositionDelete.java @@ -31,6 +31,13 @@ public static PositionDelete create() { private PositionDelete() {} + public PositionDelete set(CharSequence newPath, long newPos) { + this.path = newPath; + this.pos = newPos; + this.row = null; + return this; + } + public PositionDelete set(CharSequence newPath, long newPos, R newRow) { this.path = newPath; this.pos = newPos; diff --git a/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteIndex.java b/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteIndex.java index be05875aeb2a..3655b8b7e8eb 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteIndex.java +++ b/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteIndex.java @@ -18,6 +18,11 @@ */ package org.apache.iceberg.deletes; +import java.util.Collection; +import java.util.function.LongConsumer; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; + public interface PositionDeleteIndex { /** * Set a deleted row position. @@ -34,6 +39,18 @@ public interface PositionDeleteIndex { */ void delete(long posStart, long posEnd); + /** + * Adds positions from the other index, modifying this index in place. + * + * @param that the other index to merge + */ + default void merge(PositionDeleteIndex that) { + if (!that.deleteFiles().isEmpty()) { + throw new UnsupportedOperationException(getClass().getName() + " does not support merge"); + } + that.forEach(this::delete); + } + /** * Checks whether a row at the position is deleted. * @@ -50,6 +67,26 @@ default boolean isNotEmpty() { return !isEmpty(); } + /** + * Traverses all positions in the index in ascending order, applying the provided consumer. + * + * @param consumer a consumer for the positions + */ + default void forEach(LongConsumer consumer) { + if (isNotEmpty()) { + throw new UnsupportedOperationException(getClass().getName() + " does not support forEach"); + } + } + + /** + * Returns delete files that this index was created from or an empty collection if unknown. + * + * @return delete files that this index was created from + */ + default Collection deleteFiles() { + return ImmutableList.of(); + } + /** Returns an empty immutable position delete index. */ static PositionDeleteIndex empty() { return EmptyPositionDeleteIndex.get(); diff --git a/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteIndexUtil.java b/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteIndexUtil.java index 0c3bff28ee6b..7601232ad24f 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteIndexUtil.java +++ b/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteIndexUtil.java @@ -18,25 +18,13 @@ */ package org.apache.iceberg.deletes; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - public class PositionDeleteIndexUtil { private PositionDeleteIndexUtil() {} public static PositionDeleteIndex merge(Iterable indexes) { BitmapPositionDeleteIndex result = new BitmapPositionDeleteIndex(); - - for (PositionDeleteIndex index : indexes) { - if (index.isNotEmpty()) { - Preconditions.checkArgument( - index instanceof BitmapPositionDeleteIndex, - "Can merge only bitmap-based indexes, got %s", - index.getClass().getName()); - result.merge((BitmapPositionDeleteIndex) index); - } - } - + indexes.forEach(result::merge); return result; } } diff --git a/core/src/main/java/org/apache/iceberg/deletes/RoaringPositionBitmap.java b/core/src/main/java/org/apache/iceberg/deletes/RoaringPositionBitmap.java new file mode 100644 index 000000000000..eec130743d85 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/deletes/RoaringPositionBitmap.java @@ -0,0 +1,318 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.deletes; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.List; +import java.util.function.LongConsumer; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.roaringbitmap.RoaringBitmap; + +/** + * A bitmap that supports positive 64-bit positions (the most significant bit must be 0), but is + * optimized for cases where most positions fit in 32 bits by using an array of 32-bit Roaring + * bitmaps. The internal bitmap array is grown as needed to accommodate the largest position. + * + *

Incoming 64-bit positions are divided into a 32-bit "key" using the most significant 4 bytes + * and a 32-bit position using the least significant 4 bytes. For each key in the set of positions, + * a 32-bit Roaring bitmap is maintained to store a set of 32-bit positions for that key. + * + *

To test whether a certain position is set, its most significant 4 bytes (the key) are used to + * find a 32-bit bitmap and the least significant 4 bytes are tested for inclusion in the bitmap. If + * a bitmap is not found for the key, then the position is not set. + * + *

Positions must range from 0 (inclusive) to {@link #MAX_POSITION} (inclusive). This class + * cannot handle positions with the key equal to Integer.MAX_VALUE because the length of the + * internal bitmap array is a signed 32-bit integer, which must be greater than or equal to 0. + * Supporting Integer.MAX_VALUE as a key would require allocating a bitmap array with size + * Integer.MAX_VALUE + 1, triggering an integer overflow. + */ +class RoaringPositionBitmap { + + static final long MAX_POSITION = toPosition(Integer.MAX_VALUE - 1, Integer.MIN_VALUE); + private static final RoaringBitmap[] EMPTY_BITMAP_ARRAY = new RoaringBitmap[0]; + private static final long BITMAP_COUNT_SIZE_BYTES = 8L; + private static final long BITMAP_KEY_SIZE_BYTES = 4L; + + private RoaringBitmap[] bitmaps; + + RoaringPositionBitmap() { + this.bitmaps = EMPTY_BITMAP_ARRAY; + } + + private RoaringPositionBitmap(RoaringBitmap[] bitmaps) { + this.bitmaps = bitmaps; + } + + /** + * Sets a position in the bitmap. + * + * @param pos the position + */ + public void set(long pos) { + validatePosition(pos); + int key = key(pos); + int pos32Bits = pos32Bits(pos); + allocateBitmapsIfNeeded(key + 1 /* required bitmap array length */); + bitmaps[key].add(pos32Bits); + } + + /** + * Sets a range of positions in the bitmap. + * + * @param posStartInclusive the start position of the range (inclusive) + * @param posEndExclusive the end position of the range (exclusive) + */ + public void setRange(long posStartInclusive, long posEndExclusive) { + for (long pos = posStartInclusive; pos < posEndExclusive; pos++) { + set(pos); + } + } + + /** + * Sets all positions from the other bitmap in this bitmap, modifying this bitmap in place. + * + * @param that the other bitmap + */ + public void setAll(RoaringPositionBitmap that) { + allocateBitmapsIfNeeded(that.bitmaps.length); + for (int key = 0; key < that.bitmaps.length; key++) { + bitmaps[key].or(that.bitmaps[key]); + } + } + + /** + * Checks if a position is set in the bitmap. + * + * @param pos the position + * @return true if the position is set in this bitmap, false otherwise + */ + public boolean contains(long pos) { + validatePosition(pos); + int key = key(pos); + int pos32Bits = pos32Bits(pos); + return key < bitmaps.length && bitmaps[key].contains(pos32Bits); + } + + /** + * Indicates whether the bitmap has any positions set. + * + * @return true if the bitmap is empty, false otherwise + */ + public boolean isEmpty() { + return cardinality() == 0; + } + + /** + * Returns the number of set positions in the bitmap. + * + * @return the number of set positions + */ + public long cardinality() { + long cardinality = 0L; + for (RoaringBitmap bitmap : bitmaps) { + cardinality += bitmap.getLongCardinality(); + } + return cardinality; + } + + /** + * Applies run-length encoding wherever it is more space efficient. + * + * @return whether the bitmap was changed + */ + public boolean runLengthEncode() { + boolean changed = false; + for (RoaringBitmap bitmap : bitmaps) { + changed |= bitmap.runOptimize(); + } + return changed; + } + + /** + * Iterates over all positions in the bitmap. + * + * @param consumer a consumer for positions + */ + public void forEach(LongConsumer consumer) { + for (int key = 0; key < bitmaps.length; key++) { + forEach(key, bitmaps[key], consumer); + } + } + + @VisibleForTesting + int allocatedBitmapCount() { + return bitmaps.length; + } + + private void allocateBitmapsIfNeeded(int requiredLength) { + if (bitmaps.length < requiredLength) { + if (bitmaps.length == 0 && requiredLength == 1) { + this.bitmaps = new RoaringBitmap[] {new RoaringBitmap()}; + } else { + RoaringBitmap[] newBitmaps = new RoaringBitmap[requiredLength]; + System.arraycopy(bitmaps, 0, newBitmaps, 0, bitmaps.length); + for (int key = bitmaps.length; key < requiredLength; key++) { + newBitmaps[key] = new RoaringBitmap(); + } + this.bitmaps = newBitmaps; + } + } + } + + /** + * Returns the number of bytes required to serialize the bitmap. + * + * @return the serialized size in bytes + */ + public long serializedSizeInBytes() { + long size = BITMAP_COUNT_SIZE_BYTES; + for (RoaringBitmap bitmap : bitmaps) { + size += BITMAP_KEY_SIZE_BYTES + bitmap.serializedSizeInBytes(); + } + return size; + } + + /** + * Serializes the bitmap using the portable serialization format described below. + * + *

    + *
  • The number of 32-bit Roaring bitmaps, serialized as 8 bytes + *
  • For each 32-bit Roaring bitmap, ordered by unsigned comparison of the 32-bit keys: + *
      + *
    • The key stored as 4 bytes + *
    • Serialized 32-bit Roaring bitmap using the standard format + *
    + *
+ * + *

Note the byte order of the buffer must be little-endian. + * + * @param buffer the buffer to write to + * @see Roaring bitmap spec + */ + public void serialize(ByteBuffer buffer) { + validateByteOrder(buffer); + buffer.putLong(bitmaps.length); + for (int key = 0; key < bitmaps.length; key++) { + buffer.putInt(key); + bitmaps[key].serialize(buffer); + } + } + + /** + * Deserializes a bitmap from a buffer, assuming the portable serialization format. + * + * @param buffer the buffer to read from + * @return a new bitmap instance with the deserialized data + */ + public static RoaringPositionBitmap deserialize(ByteBuffer buffer) { + validateByteOrder(buffer); + + // the bitmap array may be sparse with more elements than the number of read bitmaps + int remainingBitmapCount = readBitmapCount(buffer); + List bitmaps = Lists.newArrayListWithExpectedSize(remainingBitmapCount); + int lastKey = -1; + + while (remainingBitmapCount > 0) { + int key = readKey(buffer, lastKey); + + // fill gaps as the bitmap array may be sparse + while (lastKey < key - 1) { + bitmaps.add(new RoaringBitmap()); + lastKey++; + } + + RoaringBitmap bitmap = readBitmap(buffer); + bitmaps.add(bitmap); + + lastKey = key; + remainingBitmapCount--; + } + + return new RoaringPositionBitmap(bitmaps.toArray(EMPTY_BITMAP_ARRAY)); + } + + private static void validateByteOrder(ByteBuffer buffer) { + Preconditions.checkArgument( + buffer.order() == ByteOrder.LITTLE_ENDIAN, + "Roaring bitmap serialization requires little-endian byte order"); + } + + private static int readBitmapCount(ByteBuffer buffer) { + long bitmapCount = buffer.getLong(); + Preconditions.checkArgument( + bitmapCount >= 0 && bitmapCount <= Integer.MAX_VALUE, + "Invalid bitmap count: %s", + bitmapCount); + return (int) bitmapCount; + } + + private static int readKey(ByteBuffer buffer, int lastKey) { + int key = buffer.getInt(); + Preconditions.checkArgument(key >= 0, "Invalid unsigned key: %s", key); + Preconditions.checkArgument(key <= Integer.MAX_VALUE - 1, "Key is too large: %s", key); + Preconditions.checkArgument(key > lastKey, "Keys must be sorted in ascending order"); + return key; + } + + private static RoaringBitmap readBitmap(ByteBuffer buffer) { + try { + RoaringBitmap bitmap = new RoaringBitmap(); + bitmap.deserialize(buffer); + buffer.position(buffer.position() + bitmap.serializedSizeInBytes()); + return bitmap; + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + // extracts high 32 bits from a 64-bit position (i.e. key) + private static int key(long pos) { + return (int) (pos >> 32); + } + + // extracts low 32 bits from a 64-bit position (i.e. 32-bit position) + private static int pos32Bits(long pos) { + return (int) pos; + } + + // combines high and low 32 bits into a 64-bit position + // the low 32 bits must be bit-masked to avoid sign extension + private static long toPosition(int key, int pos32Bits) { + return (((long) key) << 32) | (((long) pos32Bits) & 0xFFFFFFFFL); + } + + // iterates over 64-bit positions, reconstructing them from keys and 32-bit positions + private static void forEach(int key, RoaringBitmap bitmap, LongConsumer consumer) { + bitmap.forEach((int pos32Bits) -> consumer.accept(toPosition(key, pos32Bits))); + } + + private static void validatePosition(long pos) { + Preconditions.checkArgument( + pos >= 0 && pos <= MAX_POSITION, + "Bitmap supports positions that are >= 0 and <= %s: %s", + MAX_POSITION, + pos); + } +} diff --git a/core/src/main/java/org/apache/iceberg/deletes/SortingPositionOnlyDeleteWriter.java b/core/src/main/java/org/apache/iceberg/deletes/SortingPositionOnlyDeleteWriter.java index 1d4d131dfe6f..818529c02479 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/SortingPositionOnlyDeleteWriter.java +++ b/core/src/main/java/org/apache/iceberg/deletes/SortingPositionOnlyDeleteWriter.java @@ -21,17 +21,18 @@ import java.io.IOException; import java.util.Collection; import java.util.List; +import java.util.function.Function; import java.util.function.Supplier; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.io.DeleteWriteResult; import org.apache.iceberg.io.FileWriter; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Comparators; import org.apache.iceberg.util.CharSequenceMap; import org.apache.iceberg.util.CharSequenceSet; -import org.roaringbitmap.longlong.PeekableLongIterator; -import org.roaringbitmap.longlong.Roaring64Bitmap; +import org.apache.iceberg.util.ContentFileUtil; /** * A position delete writer that is capable of handling unordered deletes without rows. @@ -41,6 +42,11 @@ * records are not ordered by file and position as required by the spec. If the incoming deletes are * ordered by an external process, use {@link PositionDeleteWriter} instead. * + *

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

Note this writer stores only positions. It does not store deleted records. */ public class SortingPositionOnlyDeleteWriter @@ -48,7 +54,8 @@ public class SortingPositionOnlyDeleteWriter private final Supplier, DeleteWriteResult>> writers; private final DeleteGranularity granularity; - private final CharSequenceMap positionsByPath; + private final CharSequenceMap positionsByPath; + private final Function loadPreviousDeletes; private DeleteWriteResult result = null; public SortingPositionOnlyDeleteWriter(FileWriter, DeleteWriteResult> writer) { @@ -58,17 +65,26 @@ public SortingPositionOnlyDeleteWriter(FileWriter, DeleteWrite public SortingPositionOnlyDeleteWriter( Supplier, DeleteWriteResult>> writers, DeleteGranularity granularity) { + this(writers, granularity, path -> null /* no access to previous deletes */); + } + + public SortingPositionOnlyDeleteWriter( + Supplier, DeleteWriteResult>> writers, + DeleteGranularity granularity, + Function loadPreviousDeletes) { this.writers = writers; this.granularity = granularity; this.positionsByPath = CharSequenceMap.create(); + this.loadPreviousDeletes = loadPreviousDeletes; } @Override public void write(PositionDelete positionDelete) { CharSequence path = positionDelete.path(); long position = positionDelete.pos(); - Roaring64Bitmap positions = positionsByPath.computeIfAbsent(path, Roaring64Bitmap::new); - positions.add(position); + PositionDeleteIndex positions = + positionsByPath.computeIfAbsent(path, key -> new BitmapPositionDeleteIndex()); + positions.delete(position); } @Override @@ -106,14 +122,16 @@ private DeleteWriteResult writePartitionDeletes() throws IOException { private DeleteWriteResult writeFileDeletes() throws IOException { List deleteFiles = Lists.newArrayList(); CharSequenceSet referencedDataFiles = CharSequenceSet.empty(); + List rewrittenDeleteFiles = Lists.newArrayList(); for (CharSequence path : positionsByPath.keySet()) { DeleteWriteResult writeResult = writeDeletes(ImmutableList.of(path)); deleteFiles.addAll(writeResult.deleteFiles()); referencedDataFiles.addAll(writeResult.referencedDataFiles()); + rewrittenDeleteFiles.addAll(writeResult.rewrittenDeleteFiles()); } - return new DeleteWriteResult(deleteFiles, referencedDataFiles); + return new DeleteWriteResult(deleteFiles, referencedDataFiles, rewrittenDeleteFiles); } @SuppressWarnings("CollectionUndefinedEquality") @@ -123,22 +141,38 @@ private DeleteWriteResult writeDeletes(Collection paths) throws IO } FileWriter, DeleteWriteResult> writer = writers.get(); + List rewrittenDeleteFiles = Lists.newArrayList(); try { PositionDelete positionDelete = PositionDelete.create(); for (CharSequence path : sort(paths)) { - // the iterator provides values in ascending sorted order - PeekableLongIterator positions = positionsByPath.get(path).getLongIterator(); - while (positions.hasNext()) { - long position = positions.next(); - writer.write(positionDelete.set(path, position, null /* no row */)); + PositionDeleteIndex positions = positionsByPath.get(path); + PositionDeleteIndex previousPositions = loadPreviousDeletes.apply(path); + if (previousPositions != null && previousPositions.isNotEmpty()) { + validatePreviousDeletes(previousPositions); + positions.merge(previousPositions); + rewrittenDeleteFiles.addAll(previousPositions.deleteFiles()); } + positions.forEach(position -> writer.write(positionDelete.set(path, position))); } } finally { writer.close(); } - return writer.result(); + DeleteWriteResult writerResult = writer.result(); + List deleteFiles = writerResult.deleteFiles(); + CharSequenceSet referencedDataFiles = writerResult.referencedDataFiles(); + return new DeleteWriteResult(deleteFiles, referencedDataFiles, rewrittenDeleteFiles); + } + + private void validatePreviousDeletes(PositionDeleteIndex index) { + Preconditions.checkArgument( + index.deleteFiles().stream().allMatch(this::isFileScoped), + "Previous deletes must be file-scoped"); + } + + private boolean isFileScoped(DeleteFile deleteFile) { + return ContentFileUtil.referencedDataFile(deleteFile) != null; } private Collection sort(Collection paths) { diff --git a/core/src/main/java/org/apache/iceberg/hadoop/HadoopFileIO.java b/core/src/main/java/org/apache/iceberg/hadoop/HadoopFileIO.java index 7aaa2b6a75b1..02530cf6c9dd 100644 --- a/core/src/main/java/org/apache/iceberg/hadoop/HadoopFileIO.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopFileIO.java @@ -74,7 +74,7 @@ public HadoopFileIO(SerializableSupplier hadoopConf) { } public Configuration conf() { - return hadoopConf.get(); + return getConf(); } @Override @@ -84,23 +84,23 @@ public void initialize(Map props) { @Override public InputFile newInputFile(String path) { - return HadoopInputFile.fromLocation(path, hadoopConf.get()); + return HadoopInputFile.fromLocation(path, getConf()); } @Override public InputFile newInputFile(String path, long length) { - return HadoopInputFile.fromLocation(path, length, hadoopConf.get()); + return HadoopInputFile.fromLocation(path, length, getConf()); } @Override public OutputFile newOutputFile(String path) { - return HadoopOutputFile.fromPath(new Path(path), hadoopConf.get()); + return HadoopOutputFile.fromPath(new Path(path), getConf()); } @Override public void deleteFile(String path) { Path toDelete = new Path(path); - FileSystem fs = Util.getFs(toDelete, hadoopConf.get()); + FileSystem fs = Util.getFs(toDelete, getConf()); try { fs.delete(toDelete, false /* not recursive */); } catch (IOException e) { @@ -120,6 +120,16 @@ public void setConf(Configuration conf) { @Override public Configuration getConf() { + // Create a default hadoopConf as it is required for the object to be valid. + // E.g. newInputFile would throw NPE with getConf() otherwise. + if (hadoopConf == null) { + synchronized (this) { + if (hadoopConf == null) { + this.hadoopConf = new SerializableConfiguration(new Configuration())::get; + } + } + } + return hadoopConf.get(); } @@ -132,7 +142,7 @@ public void serializeConfWith( @Override public Iterable listPrefix(String prefix) { Path prefixToList = new Path(prefix); - FileSystem fs = Util.getFs(prefixToList, hadoopConf.get()); + FileSystem fs = Util.getFs(prefixToList, getConf()); return () -> { try { @@ -154,7 +164,7 @@ public Iterable listPrefix(String prefix) { @Override public void deletePrefix(String prefix) { Path prefixToDelete = new Path(prefix); - FileSystem fs = Util.getFs(prefixToDelete, hadoopConf.get()); + FileSystem fs = Util.getFs(prefixToDelete, getConf()); try { fs.delete(prefixToDelete, true /* recursive */); @@ -192,7 +202,8 @@ private ExecutorService executorService() { if (executorService == null) { synchronized (HadoopFileIO.class) { if (executorService == null) { - executorService = ThreadPools.newWorkerPool(DELETE_FILE_POOL_NAME, deleteThreads()); + executorService = + ThreadPools.newExitingWorkerPool(DELETE_FILE_POOL_NAME, deleteThreads()); } } } diff --git a/core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java b/core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java index 1e0cf4422120..24299371401c 100644 --- a/core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java @@ -22,7 +22,6 @@ import java.io.IOException; import java.io.InputStreamReader; import java.nio.charset.StandardCharsets; -import java.util.Set; import java.util.UUID; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -31,6 +30,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.LocationProviders; import org.apache.iceberg.LockManager; import org.apache.iceberg.TableMetadata; @@ -45,10 +45,7 @@ import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.Pair; -import org.apache.iceberg.util.Tasks; -import org.apache.iceberg.util.ThreadPools; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -168,7 +165,7 @@ public void commit(TableMetadata base, TableMetadata metadata) { // update the best-effort version pointer writeVersionHint(nextVersion); - deleteRemovedMetadataFiles(base, metadata); + CatalogUtil.deleteRemovedMetadataFiles(io(), base, metadata); this.shouldRefresh = true; } @@ -414,39 +411,6 @@ protected FileSystem getFileSystem(Path path, Configuration hadoopConf) { return Util.getFs(path, hadoopConf); } - /** - * Deletes the oldest metadata files if {@link - * TableProperties#METADATA_DELETE_AFTER_COMMIT_ENABLED} is true. - * - * @param base table metadata on which previous versions were based - * @param metadata new table metadata with updated previous versions - */ - private void deleteRemovedMetadataFiles(TableMetadata base, TableMetadata metadata) { - if (base == null) { - return; - } - - boolean deleteAfterCommit = - metadata.propertyAsBoolean( - TableProperties.METADATA_DELETE_AFTER_COMMIT_ENABLED, - TableProperties.METADATA_DELETE_AFTER_COMMIT_ENABLED_DEFAULT); - - if (deleteAfterCommit) { - Set removedPreviousMetadataFiles = - Sets.newHashSet(base.previousFiles()); - removedPreviousMetadataFiles.removeAll(metadata.previousFiles()); - Tasks.foreach(removedPreviousMetadataFiles) - .executeWith(ThreadPools.getWorkerPool()) - .noRetry() - .suppressFailureWhenFinished() - .onFailure( - (previousMetadataFile, exc) -> - LOG.warn( - "Delete failed for previous metadata file: {}", previousMetadataFile, exc)) - .run(previousMetadataFile -> io().deleteFile(previousMetadataFile.file())); - } - } - private static TableMetadata checkUUID(TableMetadata currentMetadata, TableMetadata newMetadata) { String newUUID = newMetadata.uuid(); if (currentMetadata != null && currentMetadata.uuid() != null && newUUID != null) { diff --git a/core/src/main/java/org/apache/iceberg/io/BasePositionDeltaWriter.java b/core/src/main/java/org/apache/iceberg/io/BasePositionDeltaWriter.java index e098729ba20d..859d339bd993 100644 --- a/core/src/main/java/org/apache/iceberg/io/BasePositionDeltaWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/BasePositionDeltaWriter.java @@ -81,6 +81,7 @@ public WriteResult result() { .addDataFiles(dataFiles()) .addDeleteFiles(deleteWriteResult.deleteFiles()) .addReferencedDataFiles(deleteWriteResult.referencedDataFiles()) + .addRewrittenDeleteFiles(deleteWriteResult.rewrittenDeleteFiles()) .build(); } diff --git a/core/src/main/java/org/apache/iceberg/io/ContentCache.java b/core/src/main/java/org/apache/iceberg/io/ContentCache.java index ce37cfb08934..fd171f7806e2 100644 --- a/core/src/main/java/org/apache/iceberg/io/ContentCache.java +++ b/core/src/main/java/org/apache/iceberg/io/ContentCache.java @@ -28,6 +28,7 @@ import java.nio.ByteBuffer; import java.time.Duration; import java.util.List; +import java.util.Locale; import org.apache.iceberg.exceptions.NotFoundException; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; @@ -128,10 +129,23 @@ public InputFile tryCache(InputFile input) { return input; } + /** + * Invalidate the cache entry for the given key. + * + *

Note: if there is ongoing load, this is a blocking operation, i.e. it will wait for the load + * to complete before invalidating the entry. + */ public void invalidate(String key) { cache.invalidate(key); } + /** + * @deprecated since 1.7.0, will be removed in 2.0.0; This method does only best-effort + * invalidation and is susceptible to a race condition. If the caller changed the state that + * could be cached (perhaps files on the storage) and calls this method, there is no guarantee + * that the cache will not contain stale entries some time after this method returns. + */ + @Deprecated public void invalidateAll() { cache.invalidateAll(); } @@ -253,8 +267,10 @@ private static FileContent download(InputFile input) { // IOException and let the caller fallback to non-caching input file. throw new IOException( String.format( + Locale.ROOT, "Failed to read %d bytes: %d bytes in stream", - fileLength, fileLength - totalBytesToRead)); + fileLength, + fileLength - totalBytesToRead)); } else { buffers.add(ByteBuffer.wrap(buf)); } diff --git a/core/src/main/java/org/apache/iceberg/io/DeleteWriteResult.java b/core/src/main/java/org/apache/iceberg/io/DeleteWriteResult.java index 5c8768dcf16e..98ac76bc8060 100644 --- a/core/src/main/java/org/apache/iceberg/io/DeleteWriteResult.java +++ b/core/src/main/java/org/apache/iceberg/io/DeleteWriteResult.java @@ -32,25 +32,39 @@ public class DeleteWriteResult { private final List deleteFiles; private final CharSequenceSet referencedDataFiles; + private final List rewrittenDeleteFiles; public DeleteWriteResult(DeleteFile deleteFile) { this.deleteFiles = Collections.singletonList(deleteFile); this.referencedDataFiles = CharSequenceSet.empty(); + this.rewrittenDeleteFiles = Collections.emptyList(); } public DeleteWriteResult(DeleteFile deleteFile, CharSequenceSet referencedDataFiles) { this.deleteFiles = Collections.singletonList(deleteFile); this.referencedDataFiles = referencedDataFiles; + this.rewrittenDeleteFiles = Collections.emptyList(); } public DeleteWriteResult(List deleteFiles) { this.deleteFiles = deleteFiles; this.referencedDataFiles = CharSequenceSet.empty(); + this.rewrittenDeleteFiles = Collections.emptyList(); } public DeleteWriteResult(List deleteFiles, CharSequenceSet referencedDataFiles) { this.deleteFiles = deleteFiles; this.referencedDataFiles = referencedDataFiles; + this.rewrittenDeleteFiles = Collections.emptyList(); + } + + public DeleteWriteResult( + List deleteFiles, + CharSequenceSet referencedDataFiles, + List rewrittenDeleteFiles) { + this.deleteFiles = deleteFiles; + this.referencedDataFiles = referencedDataFiles; + this.rewrittenDeleteFiles = rewrittenDeleteFiles; } public List deleteFiles() { @@ -64,4 +78,8 @@ public CharSequenceSet referencedDataFiles() { public boolean referencesDataFiles() { return referencedDataFiles != null && !referencedDataFiles.isEmpty(); } + + public List rewrittenDeleteFiles() { + return rewrittenDeleteFiles; + } } diff --git a/core/src/main/java/org/apache/iceberg/io/FanoutPositionOnlyDeleteWriter.java b/core/src/main/java/org/apache/iceberg/io/FanoutPositionOnlyDeleteWriter.java index c6a55064b756..9c527f4b32e5 100644 --- a/core/src/main/java/org/apache/iceberg/io/FanoutPositionOnlyDeleteWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/FanoutPositionOnlyDeleteWriter.java @@ -19,11 +19,13 @@ package org.apache.iceberg.io; import java.util.List; +import java.util.function.Function; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.StructLike; import org.apache.iceberg.deletes.DeleteGranularity; import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.deletes.PositionDeleteIndex; import org.apache.iceberg.deletes.SortingPositionOnlyDeleteWriter; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.CharSequenceSet; @@ -45,6 +47,8 @@ public class FanoutPositionOnlyDeleteWriter private final DeleteGranularity granularity; private final List deleteFiles; private final CharSequenceSet referencedDataFiles; + private final List rewrittenDeleteFiles; + private final Function loadPreviousDeletes; public FanoutPositionOnlyDeleteWriter( FileWriterFactory writerFactory, @@ -60,6 +64,22 @@ public FanoutPositionOnlyDeleteWriter( FileIO io, long targetFileSizeInBytes, DeleteGranularity granularity) { + this( + writerFactory, + fileFactory, + io, + targetFileSizeInBytes, + granularity, + path -> null /* no access to previous deletes */); + } + + public FanoutPositionOnlyDeleteWriter( + FileWriterFactory writerFactory, + OutputFileFactory fileFactory, + FileIO io, + long targetFileSizeInBytes, + DeleteGranularity granularity, + Function loadPreviousDeletes) { this.writerFactory = writerFactory; this.fileFactory = fileFactory; this.io = io; @@ -67,6 +87,8 @@ public FanoutPositionOnlyDeleteWriter( this.granularity = granularity; this.deleteFiles = Lists.newArrayList(); this.referencedDataFiles = CharSequenceSet.empty(); + this.rewrittenDeleteFiles = Lists.newArrayList(); + this.loadPreviousDeletes = loadPreviousDeletes; } @Override @@ -76,17 +98,19 @@ protected FileWriter, DeleteWriteResult> newWriter( () -> new RollingPositionDeleteWriter<>( writerFactory, fileFactory, io, targetFileSizeInBytes, spec, partition), - granularity); + granularity, + loadPreviousDeletes); } @Override protected void addResult(DeleteWriteResult result) { deleteFiles.addAll(result.deleteFiles()); referencedDataFiles.addAll(result.referencedDataFiles()); + rewrittenDeleteFiles.addAll(result.rewrittenDeleteFiles()); } @Override protected DeleteWriteResult aggregatedResult() { - return new DeleteWriteResult(deleteFiles, referencedDataFiles); + return new DeleteWriteResult(deleteFiles, referencedDataFiles, rewrittenDeleteFiles); } } diff --git a/core/src/main/java/org/apache/iceberg/io/OutputFileFactory.java b/core/src/main/java/org/apache/iceberg/io/OutputFileFactory.java index c46ca132ed0b..50e84143ffb7 100644 --- a/core/src/main/java/org/apache/iceberg/io/OutputFileFactory.java +++ b/core/src/main/java/org/apache/iceberg/io/OutputFileFactory.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT; +import java.util.Locale; import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Supplier; @@ -92,6 +93,7 @@ public static Builder builderFor(Table table, int partitionId, long taskId) { private String generateFilename() { return format.addExtension( String.format( + Locale.ROOT, "%05d-%d-%s-%05d%s", partitionId, taskId, diff --git a/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java b/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java index b5232960dcfb..a858045aab8b 100644 --- a/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java +++ b/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java @@ -21,8 +21,10 @@ import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Function; +import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.CatalogUtil; @@ -141,7 +143,7 @@ public void close() { @Override public void serializeConfWith( Function> confSerializer) { - this.hadoopConf = confSerializer.apply(hadoopConf.get()); + this.hadoopConf = confSerializer.apply(getConf()); } @Override @@ -151,7 +153,7 @@ public void setConf(Configuration conf) { @Override public Configuration getConf() { - return hadoopConf.get(); + return Optional.ofNullable(hadoopConf).map(Supplier::get).orElse(null); } @VisibleForTesting @@ -163,7 +165,7 @@ DelegateFileIO io(String location) { synchronized (io) { if (((HadoopConfigurable) io).getConf() == null) { // re-apply the config in case it's null after Kryo serialization - ((HadoopConfigurable) io).setConf(hadoopConf.get()); + ((HadoopConfigurable) io).setConf(getConf()); } } } @@ -174,7 +176,7 @@ DelegateFileIO io(String location) { return ioInstances.computeIfAbsent( impl, key -> { - Configuration conf = hadoopConf.get(); + Configuration conf = getConf(); FileIO fileIO; try { diff --git a/core/src/main/java/org/apache/iceberg/io/WriteResult.java b/core/src/main/java/org/apache/iceberg/io/WriteResult.java index c73b09f46c53..39efaec7d34a 100644 --- a/core/src/main/java/org/apache/iceberg/io/WriteResult.java +++ b/core/src/main/java/org/apache/iceberg/io/WriteResult.java @@ -31,12 +31,17 @@ public class WriteResult implements Serializable { private DataFile[] dataFiles; private DeleteFile[] deleteFiles; private CharSequence[] referencedDataFiles; + private DeleteFile[] rewrittenDeleteFiles; private WriteResult( - List dataFiles, List deleteFiles, CharSequenceSet referencedDataFiles) { + List dataFiles, + List deleteFiles, + CharSequenceSet referencedDataFiles, + List rewrittenDeleteFiles) { this.dataFiles = dataFiles.toArray(new DataFile[0]); this.deleteFiles = deleteFiles.toArray(new DeleteFile[0]); this.referencedDataFiles = referencedDataFiles.toArray(new CharSequence[0]); + this.rewrittenDeleteFiles = rewrittenDeleteFiles.toArray(new DeleteFile[0]); } public DataFile[] dataFiles() { @@ -51,6 +56,10 @@ public CharSequence[] referencedDataFiles() { return referencedDataFiles; } + public DeleteFile[] rewrittenDeleteFiles() { + return rewrittenDeleteFiles; + } + public static Builder builder() { return new Builder(); } @@ -59,18 +68,20 @@ public static class Builder { private final List dataFiles; private final List deleteFiles; private final CharSequenceSet referencedDataFiles; + private final List rewrittenDeleteFiles; private Builder() { this.dataFiles = Lists.newArrayList(); this.deleteFiles = Lists.newArrayList(); this.referencedDataFiles = CharSequenceSet.empty(); + this.rewrittenDeleteFiles = Lists.newArrayList(); } public Builder add(WriteResult result) { addDataFiles(result.dataFiles); addDeleteFiles(result.deleteFiles); addReferencedDataFiles(result.referencedDataFiles); - + addRewrittenDeleteFiles(result.rewrittenDeleteFiles); return this; } @@ -109,8 +120,18 @@ public Builder addReferencedDataFiles(Iterable files) { return this; } + public Builder addRewrittenDeleteFiles(DeleteFile... files) { + Collections.addAll(rewrittenDeleteFiles, files); + return this; + } + + public Builder addRewrittenDeleteFiles(Iterable files) { + Iterables.addAll(rewrittenDeleteFiles, files); + return this; + } + public WriteResult build() { - return new WriteResult(dataFiles, deleteFiles, referencedDataFiles); + return new WriteResult(dataFiles, deleteFiles, referencedDataFiles, rewrittenDeleteFiles); } } } diff --git a/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java index 5e985faafbed..1cce0e2ce77e 100644 --- a/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java +++ b/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java @@ -31,6 +31,7 @@ import java.util.AbstractMap; import java.util.Arrays; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Set; import java.util.function.Consumer; @@ -798,7 +799,11 @@ private boolean insertProperties(Namespace namespace, Map proper } throw new IllegalStateException( - String.format("Failed to insert: %d of %d succeeded", insertedRecords, properties.size())); + String.format( + Locale.ROOT, + "Failed to insert: %d of %d succeeded", + insertedRecords, + properties.size())); } private boolean updateProperties(Namespace namespace, Map properties) { @@ -818,7 +823,11 @@ private boolean updateProperties(Namespace namespace, Map proper } throw new IllegalStateException( - String.format("Failed to update: %d of %d succeeded", updatedRecords, properties.size())); + String.format( + Locale.ROOT, + "Failed to update: %d of %d succeeded", + updatedRecords, + properties.size())); } private boolean deleteProperties(Namespace namespace, Set properties) { diff --git a/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java b/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java index f1b7aa32d679..76fafe48f5b6 100644 --- a/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java +++ b/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java @@ -25,6 +25,7 @@ import java.time.OffsetDateTime; import java.time.ZoneOffset; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; @@ -71,6 +72,7 @@ import org.apache.iceberg.rest.responses.LoadTableResponse; import org.apache.iceberg.rest.responses.LoadViewResponse; import org.apache.iceberg.rest.responses.UpdateNamespacePropertiesResponse; +import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.Tasks; import org.apache.iceberg.view.BaseView; import org.apache.iceberg.view.SQLViewRepresentation; @@ -82,7 +84,7 @@ public class CatalogHandlers { private static final Schema EMPTY_SCHEMA = new Schema(); - private static final String INTIAL_PAGE_TOKEN = ""; + private static final String INITIAL_PAGE_TOKEN = ""; private CatalogHandlers() {} @@ -108,6 +110,19 @@ public CommitFailedException wrapped() { } } + private static Pair, String> paginate(List list, String pageToken, int pageSize) { + int pageStart = INITIAL_PAGE_TOKEN.equals(pageToken) ? 0 : Integer.parseInt(pageToken); + if (pageStart >= list.size()) { + return Pair.of(Collections.emptyList(), null); + } + + int end = Math.min(pageStart + pageSize, list.size()); + List subList = list.subList(pageStart, end); + String nextPageToken = end >= list.size() ? null : String.valueOf(end); + + return Pair.of(subList, nextPageToken); + } + public static ListNamespacesResponse listNamespaces( SupportsNamespaces catalog, Namespace parent) { List results; @@ -123,7 +138,6 @@ public static ListNamespacesResponse listNamespaces( public static ListNamespacesResponse listNamespaces( SupportsNamespaces catalog, Namespace parent, String pageToken, String pageSize) { List results; - List subResults; if (parent.isEmpty()) { results = catalog.listNamespaces(); @@ -131,16 +145,12 @@ public static ListNamespacesResponse listNamespaces( results = catalog.listNamespaces(parent); } - int start = INTIAL_PAGE_TOKEN.equals(pageToken) ? 0 : Integer.parseInt(pageToken); - int end = start + Integer.parseInt(pageSize); - subResults = results.subList(start, end); - String nextToken = String.valueOf(end); - - if (end >= results.size()) { - nextToken = null; - } + Pair, String> page = paginate(results, pageToken, Integer.parseInt(pageSize)); - return ListNamespacesResponse.builder().addAll(subResults).nextPageToken(nextToken).build(); + return ListNamespacesResponse.builder() + .addAll(page.first()) + .nextPageToken(page.second()) + .build(); } public static CreateNamespaceResponse createNamespace( @@ -203,18 +213,11 @@ public static ListTablesResponse listTables(Catalog catalog, Namespace namespace public static ListTablesResponse listTables( Catalog catalog, Namespace namespace, String pageToken, String pageSize) { List results = catalog.listTables(namespace); - List subResults; - - int start = INTIAL_PAGE_TOKEN.equals(pageToken) ? 0 : Integer.parseInt(pageToken); - int end = start + Integer.parseInt(pageSize); - subResults = results.subList(start, end); - String nextToken = String.valueOf(end); - if (end >= results.size()) { - nextToken = null; - } + Pair, String> page = + paginate(results, pageToken, Integer.parseInt(pageSize)); - return ListTablesResponse.builder().addAll(subResults).nextPageToken(nextToken).build(); + return ListTablesResponse.builder().addAll(page.first()).nextPageToken(page.second()).build(); } public static LoadTableResponse stageTableCreate( @@ -448,18 +451,11 @@ public static ListTablesResponse listViews(ViewCatalog catalog, Namespace namesp public static ListTablesResponse listViews( ViewCatalog catalog, Namespace namespace, String pageToken, String pageSize) { List results = catalog.listViews(namespace); - List subResults; - - int start = INTIAL_PAGE_TOKEN.equals(pageToken) ? 0 : Integer.parseInt(pageToken); - int end = start + Integer.parseInt(pageSize); - subResults = results.subList(start, end); - String nextToken = String.valueOf(end); - if (end >= results.size()) { - nextToken = null; - } + Pair, String> page = + paginate(results, pageToken, Integer.parseInt(pageSize)); - return ListTablesResponse.builder().addAll(subResults).nextPageToken(nextToken).build(); + return ListTablesResponse.builder().addAll(page.first()).nextPageToken(page.second()).build(); } public static LoadViewResponse createView( diff --git a/core/src/main/java/org/apache/iceberg/rest/Endpoint.java b/core/src/main/java/org/apache/iceberg/rest/Endpoint.java new file mode 100644 index 000000000000..2a8e6d633297 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/Endpoint.java @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest; + +import java.util.List; +import java.util.Objects; +import java.util.Set; +import org.apache.hc.core5.http.Method; +import org.apache.iceberg.relocated.com.google.common.base.Joiner; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.base.Splitter; +import org.apache.iceberg.relocated.com.google.common.base.Strings; +import org.apache.iceberg.relocated.com.google.common.base.Supplier; + +/** + * Holds an endpoint definition that consists of the HTTP method (GET, POST, DELETE, ...) and the + * resource path as defined in the Iceberg OpenAPI REST specification without parameter + * substitution, such as /v1/{prefix}/namespaces/{namespace}. + */ +public class Endpoint { + + // namespace endpoints + public static final Endpoint V1_LIST_NAMESPACES = + Endpoint.create("GET", ResourcePaths.V1_NAMESPACES); + public static final Endpoint V1_LOAD_NAMESPACE = + Endpoint.create("GET", ResourcePaths.V1_NAMESPACE); + public static final Endpoint V1_CREATE_NAMESPACE = + Endpoint.create("POST", ResourcePaths.V1_NAMESPACES); + public static final Endpoint V1_UPDATE_NAMESPACE = + Endpoint.create("POST", ResourcePaths.V1_NAMESPACE_PROPERTIES); + public static final Endpoint V1_DELETE_NAMESPACE = + Endpoint.create("DELETE", ResourcePaths.V1_NAMESPACE); + public static final Endpoint V1_COMMIT_TRANSACTION = + Endpoint.create("POST", ResourcePaths.V1_TRANSACTIONS_COMMIT); + + // table endpoints + public static final Endpoint V1_LIST_TABLES = Endpoint.create("GET", ResourcePaths.V1_TABLES); + public static final Endpoint V1_LOAD_TABLE = Endpoint.create("GET", ResourcePaths.V1_TABLE); + public static final Endpoint V1_CREATE_TABLE = Endpoint.create("POST", ResourcePaths.V1_TABLES); + public static final Endpoint V1_UPDATE_TABLE = Endpoint.create("POST", ResourcePaths.V1_TABLE); + public static final Endpoint V1_DELETE_TABLE = Endpoint.create("DELETE", ResourcePaths.V1_TABLE); + public static final Endpoint V1_RENAME_TABLE = + Endpoint.create("POST", ResourcePaths.V1_TABLE_RENAME); + public static final Endpoint V1_REGISTER_TABLE = + Endpoint.create("POST", ResourcePaths.V1_TABLE_REGISTER); + public static final Endpoint V1_REPORT_METRICS = + Endpoint.create("POST", ResourcePaths.V1_TABLE_METRICS); + + // view endpoints + public static final Endpoint V1_LIST_VIEWS = Endpoint.create("GET", ResourcePaths.V1_VIEWS); + public static final Endpoint V1_LOAD_VIEW = Endpoint.create("GET", ResourcePaths.V1_VIEW); + public static final Endpoint V1_CREATE_VIEW = Endpoint.create("POST", ResourcePaths.V1_VIEWS); + public static final Endpoint V1_UPDATE_VIEW = Endpoint.create("POST", ResourcePaths.V1_VIEW); + public static final Endpoint V1_DELETE_VIEW = Endpoint.create("DELETE", ResourcePaths.V1_VIEW); + public static final Endpoint V1_RENAME_VIEW = + Endpoint.create("POST", ResourcePaths.V1_VIEW_RENAME); + + private static final Splitter ENDPOINT_SPLITTER = Splitter.on(" "); + private static final Joiner ENDPOINT_JOINER = Joiner.on(" "); + private final String httpMethod; + private final String path; + + private Endpoint(String httpMethod, String path) { + Preconditions.checkArgument( + !Strings.isNullOrEmpty(httpMethod), "Invalid HTTP method: null or empty"); + Preconditions.checkArgument(!Strings.isNullOrEmpty(path), "Invalid path: null or empty"); + this.httpMethod = Method.normalizedValueOf(httpMethod).toString(); + this.path = path; + } + + public String httpMethod() { + return httpMethod; + } + + public String path() { + return path; + } + + public static Endpoint create(String httpMethod, String path) { + return new Endpoint(httpMethod, path); + } + + @Override + public String toString() { + return ENDPOINT_JOINER.join(httpMethod(), path()); + } + + public static Endpoint fromString(String endpoint) { + List elements = ENDPOINT_SPLITTER.splitToList(endpoint); + Preconditions.checkArgument( + elements.size() == 2, + "Invalid endpoint (must consist of two elements separated by a single space): %s", + endpoint); + return create(elements.get(0), elements.get(1)); + } + + /** + * Checks if the set of endpoints support the given {@link Endpoint}. + * + * @param supportedEndpoints The set of supported endpoints to check + * @param endpoint The endpoint to check against the set of supported endpoints + * @throws UnsupportedOperationException if the given {@link Endpoint} is not included in the set + * of endpoints. + */ + public static void check(Set supportedEndpoints, Endpoint endpoint) { + if (!supportedEndpoints.contains(endpoint)) { + throw new UnsupportedOperationException( + String.format("Server does not support endpoint: %s", endpoint)); + } + } + + /** + * Checks if the set of endpoints support the given {@link Endpoint}. + * + * @param supportedEndpoints The set of supported endpoints to check + * @param endpoint The endpoint to check against the set of supported endpoints + * @param supplier The supplier throwing a {@link RuntimeException} if the given {@link Endpoint} + * is not included in the set of endpoints. + */ + public static void check( + Set supportedEndpoints, Endpoint endpoint, Supplier supplier) { + if (!supportedEndpoints.contains(endpoint)) { + throw supplier.get(); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof Endpoint)) { + return false; + } + + Endpoint endpoint = (Endpoint) o; + return Objects.equals(httpMethod, endpoint.httpMethod) && Objects.equals(path, endpoint.path); + } + + @Override + public int hashCode() { + return Objects.hash(httpMethod, path); + } +} diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTSerializers.java b/core/src/main/java/org/apache/iceberg/rest/RESTSerializers.java index 341dda0e3f6c..667142698633 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSerializers.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSerializers.java @@ -59,7 +59,12 @@ import org.apache.iceberg.rest.responses.ConfigResponseParser; import org.apache.iceberg.rest.responses.ErrorResponse; import org.apache.iceberg.rest.responses.ErrorResponseParser; +import org.apache.iceberg.rest.responses.ImmutableLoadCredentialsResponse; import org.apache.iceberg.rest.responses.ImmutableLoadViewResponse; +import org.apache.iceberg.rest.responses.LoadCredentialsResponse; +import org.apache.iceberg.rest.responses.LoadCredentialsResponseParser; +import org.apache.iceberg.rest.responses.LoadTableResponse; +import org.apache.iceberg.rest.responses.LoadTableResponseParser; import org.apache.iceberg.rest.responses.LoadViewResponse; import org.apache.iceberg.rest.responses.LoadViewResponseParser; import org.apache.iceberg.rest.responses.OAuthTokenResponse; @@ -115,7 +120,15 @@ public static void registerAll(ObjectMapper mapper) { .addDeserializer(LoadViewResponse.class, new LoadViewResponseDeserializer<>()) .addDeserializer(ImmutableLoadViewResponse.class, new LoadViewResponseDeserializer<>()) .addSerializer(ConfigResponse.class, new ConfigResponseSerializer<>()) - .addDeserializer(ConfigResponse.class, new ConfigResponseDeserializer<>()); + .addDeserializer(ConfigResponse.class, new ConfigResponseDeserializer<>()) + .addSerializer(LoadTableResponse.class, new LoadTableResponseSerializer<>()) + .addDeserializer(LoadTableResponse.class, new LoadTableResponseDeserializer<>()) + .addSerializer(LoadCredentialsResponse.class, new LoadCredentialsResponseSerializer<>()) + .addSerializer( + ImmutableLoadCredentialsResponse.class, new LoadCredentialsResponseSerializer<>()) + .addDeserializer(LoadCredentialsResponse.class, new LoadCredentialsResponseDeserializer<>()) + .addDeserializer( + ImmutableLoadCredentialsResponse.class, new LoadCredentialsResponseDeserializer<>()); mapper.registerModule(module); } @@ -422,4 +435,39 @@ public T deserialize(JsonParser p, DeserializationContext context) throws IOExce return (T) ConfigResponseParser.fromJson(jsonNode); } } + + static class LoadTableResponseSerializer extends JsonSerializer { + @Override + public void serialize(T request, JsonGenerator gen, SerializerProvider serializers) + throws IOException { + LoadTableResponseParser.toJson(request, gen); + } + } + + static class LoadTableResponseDeserializer + extends JsonDeserializer { + @Override + public T deserialize(JsonParser p, DeserializationContext context) throws IOException { + JsonNode jsonNode = p.getCodec().readTree(p); + return (T) LoadTableResponseParser.fromJson(jsonNode); + } + } + + static class LoadCredentialsResponseSerializer + extends JsonSerializer { + @Override + public void serialize(T request, JsonGenerator gen, SerializerProvider serializers) + throws IOException { + LoadCredentialsResponseParser.toJson(request, gen); + } + } + + static class LoadCredentialsResponseDeserializer + extends JsonDeserializer { + @Override + public T deserialize(JsonParser p, DeserializationContext context) throws IOException { + JsonNode jsonNode = p.getCodec().readTree(p); + return (T) LoadCredentialsResponseParser.fromJson(jsonNode); + } + } } diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java index 53ce45bb0a3f..cc42604f700d 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -58,7 +58,6 @@ import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.exceptions.NoSuchViewException; -import org.apache.iceberg.exceptions.RESTException; import org.apache.iceberg.hadoop.Configurable; import org.apache.iceberg.io.CloseableGroup; import org.apache.iceberg.io.FileIO; @@ -116,6 +115,9 @@ public class RESTSessionCatalog extends BaseViewSessionCatalog private static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.io.ResolvingFileIO"; private static final String REST_METRICS_REPORTING_ENABLED = "rest-metrics-reporting-enabled"; private static final String REST_SNAPSHOT_LOADING_MODE = "snapshot-loading-mode"; + // for backwards compatibility with older REST servers where it can be assumed that a particular + // server supports view endpoints but doesn't send the "endpoints" field in the ConfigResponse + static final String VIEW_ENDPOINTS_SUPPORTED = "view-endpoints-supported"; public static final String REST_PAGE_SIZE = "rest-page-size"; private static final List TOKEN_PREFERENCE_ORDER = ImmutableList.of( @@ -132,6 +134,33 @@ public class RESTSessionCatalog extends BaseViewSessionCatalog .addAll(TOKEN_PREFERENCE_ORDER) .build(); + private static final Set DEFAULT_ENDPOINTS = + ImmutableSet.builder() + .add(Endpoint.V1_LIST_NAMESPACES) + .add(Endpoint.V1_LOAD_NAMESPACE) + .add(Endpoint.V1_CREATE_NAMESPACE) + .add(Endpoint.V1_UPDATE_NAMESPACE) + .add(Endpoint.V1_DELETE_NAMESPACE) + .add(Endpoint.V1_LIST_TABLES) + .add(Endpoint.V1_LOAD_TABLE) + .add(Endpoint.V1_CREATE_TABLE) + .add(Endpoint.V1_UPDATE_TABLE) + .add(Endpoint.V1_DELETE_TABLE) + .add(Endpoint.V1_RENAME_TABLE) + .add(Endpoint.V1_REGISTER_TABLE) + .add(Endpoint.V1_REPORT_METRICS) + .build(); + + private static final Set VIEW_ENDPOINTS = + ImmutableSet.builder() + .add(Endpoint.V1_LIST_VIEWS) + .add(Endpoint.V1_LOAD_VIEW) + .add(Endpoint.V1_CREATE_VIEW) + .add(Endpoint.V1_UPDATE_VIEW) + .add(Endpoint.V1_DELETE_VIEW) + .add(Endpoint.V1_RENAME_VIEW) + .build(); + private final Function, RESTClient> clientBuilder; private final BiFunction, FileIO> ioBuilder; private Cache sessions = null; @@ -148,6 +177,7 @@ public class RESTSessionCatalog extends BaseViewSessionCatalog private boolean reportingViaRestEnabled; private Integer pageSize = null; private CloseableGroup closeables = null; + private Set endpoints; // a lazy thread pool for token refresh private volatile ScheduledExecutorService refreshExecutor = null; @@ -173,6 +203,7 @@ public RESTSessionCatalog( this.ioBuilder = ioBuilder; } + @SuppressWarnings("checkstyle:CyclomaticComplexity") @Override public void initialize(String name, Map unresolved) { Preconditions.checkArgument(unresolved != null, "Invalid configuration: null"); @@ -230,6 +261,18 @@ public void initialize(String name, Map unresolved) { Map mergedProps = config.merge(props); Map baseHeaders = configHeaders(mergedProps); + if (config.endpoints().isEmpty()) { + this.endpoints = + PropertyUtil.propertyAsBoolean(mergedProps, VIEW_ENDPOINTS_SUPPORTED, false) + ? ImmutableSet.builder() + .addAll(DEFAULT_ENDPOINTS) + .addAll(VIEW_ENDPOINTS) + .build() + : DEFAULT_ENDPOINTS; + } else { + this.endpoints = ImmutableSet.copyOf(config.endpoints()); + } + this.sessions = newSessionCache(mergedProps); this.tableSessions = newSessionCache(mergedProps); this.keepTokenRefreshed = @@ -316,6 +359,10 @@ public void setConf(Object newConf) { @Override public List listTables(SessionContext context, Namespace ns) { + if (!endpoints.contains(Endpoint.V1_LIST_TABLES)) { + return ImmutableList.of(); + } + checkNamespaceIsValid(ns); Map queryParams = Maps.newHashMap(); ImmutableList.Builder tables = ImmutableList.builder(); @@ -342,6 +389,7 @@ public List listTables(SessionContext context, Namespace ns) { @Override public boolean dropTable(SessionContext context, TableIdentifier identifier) { + Endpoint.check(endpoints, Endpoint.V1_DELETE_TABLE); checkIdentifierIsValid(identifier); try { @@ -355,6 +403,7 @@ public boolean dropTable(SessionContext context, TableIdentifier identifier) { @Override public boolean purgeTable(SessionContext context, TableIdentifier identifier) { + Endpoint.check(endpoints, Endpoint.V1_DELETE_TABLE); checkIdentifierIsValid(identifier); try { @@ -372,6 +421,7 @@ public boolean purgeTable(SessionContext context, TableIdentifier identifier) { @Override public void renameTable(SessionContext context, TableIdentifier from, TableIdentifier to) { + Endpoint.check(endpoints, Endpoint.V1_RENAME_TABLE); checkIdentifierIsValid(from); checkIdentifierIsValid(to); @@ -384,6 +434,7 @@ public void renameTable(SessionContext context, TableIdentifier from, TableIdent private LoadTableResponse loadInternal( SessionContext context, TableIdentifier identifier, SnapshotMode mode) { + Endpoint.check(endpoints, Endpoint.V1_LOAD_TABLE); return client.get( paths.table(identifier), mode.params(), @@ -394,6 +445,14 @@ private LoadTableResponse loadInternal( @Override public Table loadTable(SessionContext context, TableIdentifier identifier) { + Endpoint.check( + endpoints, + Endpoint.V1_LOAD_TABLE, + () -> + new NoSuchTableException( + "Unable to load table %s.%s: Server does not support endpoint %s", + name(), identifier, Endpoint.V1_LOAD_TABLE)); + checkIdentifierIsValid(identifier); MetadataTableType metadataType; @@ -448,7 +507,8 @@ public Table loadTable(SessionContext context, TableIdentifier identifier) { paths.table(finalIdentifier), session::headers, tableFileIO(context, response.config()), - tableMetadata); + tableMetadata, + endpoints); trackFileIO(ops); @@ -472,7 +532,7 @@ private void trackFileIO(RESTTableOperations ops) { private MetricsReporter metricsReporter( String metricsEndpoint, Supplier> headers) { - if (reportingViaRestEnabled) { + if (reportingViaRestEnabled && endpoints.contains(Endpoint.V1_REPORT_METRICS)) { RESTMetricsReporter restMetricsReporter = new RESTMetricsReporter(client, metricsEndpoint, headers); return MetricsReporters.combine(reporter, restMetricsReporter); @@ -493,6 +553,7 @@ public void invalidateTable(SessionContext context, TableIdentifier ident) {} @Override public Table registerTable( SessionContext context, TableIdentifier ident, String metadataFileLocation) { + Endpoint.check(endpoints, Endpoint.V1_REGISTER_TABLE); checkIdentifierIsValid(ident); Preconditions.checkArgument( @@ -521,7 +582,8 @@ public Table registerTable( paths.table(ident), session::headers, tableFileIO(context, response.config()), - response.tableMetadata()); + response.tableMetadata(), + endpoints); trackFileIO(ops); @@ -532,6 +594,7 @@ public Table registerTable( @Override public void createNamespace( SessionContext context, Namespace namespace, Map metadata) { + Endpoint.check(endpoints, Endpoint.V1_CREATE_NAMESPACE); CreateNamespaceRequest request = CreateNamespaceRequest.builder().withNamespace(namespace).setProperties(metadata).build(); @@ -546,6 +609,10 @@ public void createNamespace( @Override public List listNamespaces(SessionContext context, Namespace namespace) { + if (!endpoints.contains(Endpoint.V1_LIST_NAMESPACES)) { + return ImmutableList.of(); + } + Map queryParams = Maps.newHashMap(); if (!namespace.isEmpty()) { queryParams.put("parent", RESTUtil.encodeNamespace(namespace)); @@ -575,6 +642,7 @@ public List listNamespaces(SessionContext context, Namespace namespac @Override public Map loadNamespaceMetadata(SessionContext context, Namespace ns) { + Endpoint.check(endpoints, Endpoint.V1_LOAD_NAMESPACE); checkNamespaceIsValid(ns); // TODO: rename to LoadNamespaceResponse? @@ -589,6 +657,7 @@ public Map loadNamespaceMetadata(SessionContext context, Namespa @Override public boolean dropNamespace(SessionContext context, Namespace ns) { + Endpoint.check(endpoints, Endpoint.V1_DELETE_NAMESPACE); checkNamespaceIsValid(ns); try { @@ -603,6 +672,7 @@ public boolean dropNamespace(SessionContext context, Namespace ns) { @Override public boolean updateNamespaceMetadata( SessionContext context, Namespace ns, Map updates, Set removals) { + Endpoint.check(endpoints, Endpoint.V1_UPDATE_NAMESPACE); checkNamespaceIsValid(ns); UpdateNamespacePropertiesRequest request = @@ -719,6 +789,7 @@ public Builder withProperty(String key, String value) { @Override public Table create() { + Endpoint.check(endpoints, Endpoint.V1_CREATE_TABLE); CreateTableRequest request = CreateTableRequest.builder() .withName(ident.name()) @@ -744,7 +815,8 @@ public Table create() { paths.table(ident), session::headers, tableFileIO(context, response.config()), - response.tableMetadata()); + response.tableMetadata(), + endpoints); trackFileIO(ops); @@ -754,6 +826,7 @@ public Table create() { @Override public Transaction createTransaction() { + Endpoint.check(endpoints, Endpoint.V1_CREATE_TABLE); LoadTableResponse response = stageCreate(); String fullName = fullTableName(ident); @@ -768,7 +841,8 @@ public Transaction createTransaction() { tableFileIO(context, response.config()), RESTTableOperations.UpdateType.CREATE, createChanges(meta), - meta); + meta, + endpoints); trackFileIO(ops); @@ -778,15 +852,9 @@ public Transaction createTransaction() { @Override public Transaction replaceTransaction() { - try { - if (viewExists(context, ident)) { - throw new AlreadyExistsException("View with same name already exists: %s", ident); - } - } catch (RESTException | UnsupportedOperationException e) { - // don't fail if the server doesn't support views, which could be due to: - // 1. server or backing catalog doesn't support views - // 2. newer client talks to an older server that doesn't support views - LOG.debug("Failed to check whether view {} exists", ident, e); + Endpoint.check(endpoints, Endpoint.V1_UPDATE_TABLE); + if (viewExists(context, ident)) { + throw new AlreadyExistsException("View with same name already exists: %s", ident); } LoadTableResponse response = loadInternal(context, ident, snapshotMode); @@ -832,7 +900,8 @@ public Transaction replaceTransaction() { tableFileIO(context, response.config()), RESTTableOperations.UpdateType.REPLACE, changes.build(), - base); + base, + endpoints); trackFileIO(ops); @@ -1085,6 +1154,7 @@ private static Cache newSessionCache(Map pr } public void commitTransaction(SessionContext context, List commits) { + Endpoint.check(endpoints, Endpoint.V1_COMMIT_TRANSACTION); List tableChanges = Lists.newArrayListWithCapacity(commits.size()); for (TableCommit commit : commits) { @@ -1102,6 +1172,10 @@ public void commitTransaction(SessionContext context, List commits) @Override public List listViews(SessionContext context, Namespace namespace) { + if (!endpoints.contains(Endpoint.V1_LIST_VIEWS)) { + return ImmutableList.of(); + } + checkNamespaceIsValid(namespace); Map queryParams = Maps.newHashMap(); ImmutableList.Builder views = ImmutableList.builder(); @@ -1128,30 +1202,29 @@ public List listViews(SessionContext context, Namespace namespa @Override public View loadView(SessionContext context, TableIdentifier identifier) { + Endpoint.check( + endpoints, + Endpoint.V1_LOAD_VIEW, + () -> + new NoSuchViewException( + "Unable to load view %s.%s: Server does not support endpoint %s", + name(), identifier, Endpoint.V1_LOAD_VIEW)); + checkViewIdentifierIsValid(identifier); - LoadViewResponse response; - try { - response = - client.get( - paths.view(identifier), - LoadViewResponse.class, - headers(context), - ErrorHandlers.viewErrorHandler()); - } catch (UnsupportedOperationException | RESTException e) { - // Normally, copying an exception message is a bad practice but engines may show just the - // message and suppress the exception cause when the view does not exist. Since 401 and 403 - // responses can trigger this case, including the message increases the chances that the "Not - // authorized" or "Forbidden" message is preserved and shown. - throw new NoSuchViewException( - e, "Unable to load view %s.%s: %s", name(), identifier, e.getMessage()); - } + LoadViewResponse response = + client.get( + paths.view(identifier), + LoadViewResponse.class, + headers(context), + ErrorHandlers.viewErrorHandler()); AuthSession session = tableSession(response.config(), session(context)); ViewMetadata metadata = response.metadata(); RESTViewOperations ops = - new RESTViewOperations(client, paths.view(identifier), session::headers, metadata); + new RESTViewOperations( + client, paths.view(identifier), session::headers, metadata, endpoints); return new BaseView(ops, ViewUtil.fullViewName(name(), identifier)); } @@ -1163,6 +1236,7 @@ public RESTViewBuilder buildView(SessionContext context, TableIdentifier identif @Override public boolean dropView(SessionContext context, TableIdentifier identifier) { + Endpoint.check(endpoints, Endpoint.V1_DELETE_VIEW); checkViewIdentifierIsValid(identifier); try { @@ -1176,6 +1250,7 @@ public boolean dropView(SessionContext context, TableIdentifier identifier) { @Override public void renameView(SessionContext context, TableIdentifier from, TableIdentifier to) { + Endpoint.check(endpoints, Endpoint.V1_RENAME_VIEW); checkViewIdentifierIsValid(from); checkViewIdentifierIsValid(to); @@ -1247,6 +1322,7 @@ public ViewBuilder withLocation(String newLocation) { @Override public View create() { + Endpoint.check(endpoints, Endpoint.V1_CREATE_VIEW); Preconditions.checkState( !representations.isEmpty(), "Cannot create view without specifying a query"); Preconditions.checkState(null != schema, "Cannot create view without specifying schema"); @@ -1284,7 +1360,7 @@ public View create() { AuthSession session = tableSession(response.config(), session(context)); RESTViewOperations ops = new RESTViewOperations( - client, paths.view(identifier), session::headers, response.metadata()); + client, paths.view(identifier), session::headers, response.metadata(), endpoints); return new BaseView(ops, ViewUtil.fullViewName(name(), identifier)); } @@ -1308,6 +1384,14 @@ public View replace() { } private LoadViewResponse loadView() { + Endpoint.check( + endpoints, + Endpoint.V1_LOAD_VIEW, + () -> + new NoSuchViewException( + "Unable to load view %s.%s: Server does not support endpoint %s", + name(), identifier, Endpoint.V1_LOAD_VIEW)); + return client.get( paths.view(identifier), LoadViewResponse.class, @@ -1316,6 +1400,7 @@ private LoadViewResponse loadView() { } private View replace(LoadViewResponse response) { + Endpoint.check(endpoints, Endpoint.V1_UPDATE_VIEW); Preconditions.checkState( !representations.isEmpty(), "Cannot replace view without specifying a query"); Preconditions.checkState(null != schema, "Cannot replace view without specifying schema"); @@ -1354,7 +1439,8 @@ private View replace(LoadViewResponse response) { AuthSession session = tableSession(response.config(), session(context)); RESTViewOperations ops = - new RESTViewOperations(client, paths.view(identifier), session::headers, metadata); + new RESTViewOperations( + client, paths.view(identifier), session::headers, metadata, endpoints); ops.commit(metadata, replacement); diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTTableOperations.java b/core/src/main/java/org/apache/iceberg/rest/RESTTableOperations.java index 0ce1afd93a79..5f6c28b32337 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTTableOperations.java @@ -21,6 +21,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Set; import java.util.function.Consumer; import java.util.function.Supplier; import org.apache.iceberg.LocationProviders; @@ -56,6 +57,7 @@ enum UpdateType { private final FileIO io; private final List createChanges; private final TableMetadata replaceBase; + private final Set endpoints; private UpdateType updateType; private TableMetadata current; @@ -64,8 +66,9 @@ enum UpdateType { String path, Supplier> headers, FileIO io, - TableMetadata current) { - this(client, path, headers, io, UpdateType.SIMPLE, Lists.newArrayList(), current); + TableMetadata current, + Set endpoints) { + this(client, path, headers, io, UpdateType.SIMPLE, Lists.newArrayList(), current, endpoints); } RESTTableOperations( @@ -75,7 +78,8 @@ enum UpdateType { FileIO io, UpdateType updateType, List createChanges, - TableMetadata current) { + TableMetadata current, + Set endpoints) { this.client = client; this.path = path; this.headers = headers; @@ -88,6 +92,7 @@ enum UpdateType { } else { this.current = current; } + this.endpoints = endpoints; } @Override @@ -97,12 +102,14 @@ public TableMetadata current() { @Override public TableMetadata refresh() { + Endpoint.check(endpoints, Endpoint.V1_LOAD_TABLE); return updateCurrentMetadata( client.get(path, LoadTableResponse.class, headers, ErrorHandlers.tableErrorHandler())); } @Override public void commit(TableMetadata base, TableMetadata metadata) { + Endpoint.check(endpoints, Endpoint.V1_UPDATE_TABLE); Consumer errorHandler; List requirements; List updates; diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTViewOperations.java b/core/src/main/java/org/apache/iceberg/rest/RESTViewOperations.java index b4dafaa9031b..466a8e66899b 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTViewOperations.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTViewOperations.java @@ -20,6 +20,7 @@ import java.util.Map; import java.util.Objects; +import java.util.Set; import java.util.function.Supplier; import org.apache.iceberg.UpdateRequirements; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -32,15 +33,21 @@ class RESTViewOperations implements ViewOperations { private final RESTClient client; private final String path; private final Supplier> headers; + private final Set endpoints; private ViewMetadata current; RESTViewOperations( - RESTClient client, String path, Supplier> headers, ViewMetadata current) { + RESTClient client, + String path, + Supplier> headers, + ViewMetadata current, + Set endpoints) { Preconditions.checkArgument(null != current, "Invalid view metadata: null"); this.client = client; this.path = path; this.headers = headers; this.current = current; + this.endpoints = endpoints; } @Override @@ -50,12 +57,14 @@ public ViewMetadata current() { @Override public ViewMetadata refresh() { + Endpoint.check(endpoints, Endpoint.V1_LOAD_VIEW); return updateCurrentMetadata( client.get(path, LoadViewResponse.class, headers, ErrorHandlers.viewErrorHandler())); } @Override public void commit(ViewMetadata base, ViewMetadata metadata) { + Endpoint.check(endpoints, Endpoint.V1_UPDATE_VIEW); // this is only used for replacing view metadata Preconditions.checkState(base != null, "Invalid base metadata: null"); diff --git a/core/src/main/java/org/apache/iceberg/rest/ResourcePaths.java b/core/src/main/java/org/apache/iceberg/rest/ResourcePaths.java index c68a4f450843..5ba7eae28262 100644 --- a/core/src/main/java/org/apache/iceberg/rest/ResourcePaths.java +++ b/core/src/main/java/org/apache/iceberg/rest/ResourcePaths.java @@ -26,6 +26,20 @@ public class ResourcePaths { private static final Joiner SLASH = Joiner.on("/").skipNulls(); private static final String PREFIX = "prefix"; + public static final String V1_NAMESPACES = "/v1/{prefix}/namespaces"; + public static final String V1_NAMESPACE = "/v1/{prefix}/namespaces/{namespace}"; + public static final String V1_NAMESPACE_PROPERTIES = + "/v1/{prefix}/namespaces/{namespace}/properties"; + public static final String V1_TABLES = "/v1/{prefix}/namespaces/{namespace}/tables"; + public static final String V1_TABLE = "/v1/{prefix}/namespaces/{namespace}/tables/{table}"; + public static final String V1_TABLE_REGISTER = "/v1/{prefix}/namespaces/{namespace}/register"; + public static final String V1_TABLE_METRICS = + "/v1/{prefix}/namespaces/{namespace}/tables/{table}/metrics"; + public static final String V1_TABLE_RENAME = "/v1/{prefix}/tables/rename"; + public static final String V1_TRANSACTIONS_COMMIT = "/v1/{prefix}/transactions/commit"; + public static final String V1_VIEWS = "/v1/{prefix}/namespaces/{namespace}/views"; + public static final String V1_VIEW = "/v1/{prefix}/namespaces/{namespace}/views/{view}"; + public static final String V1_VIEW_RENAME = "/v1/{prefix}/views/rename"; public static ResourcePaths forCatalogProperties(Map properties) { return new ResourcePaths(properties.get(PREFIX)); diff --git a/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java b/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java index 52c89af9d474..1757ae653cc9 100644 --- a/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java +++ b/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java @@ -72,7 +72,6 @@ private OAuth2Util() {} private static final String CLIENT_CREDENTIALS = "client_credentials"; private static final String TOKEN_EXCHANGE = "urn:ietf:params:oauth:grant-type:token-exchange"; private static final String SCOPE = "scope"; - private static final String CATALOG = "catalog"; // Client credentials flow private static final String CLIENT_ID = "client_id"; @@ -97,7 +96,6 @@ private OAuth2Util() {} private static final String TOKEN_TYPE = "token_type"; private static final String EXPIRES_IN = "expires_in"; private static final String ISSUED_TOKEN_TYPE = "issued_token_type"; - private static final String REFRESH_TOKEN = "refresh_token"; public static Map authHeaders(String token) { if (token != null) { diff --git a/core/src/main/java/org/apache/iceberg/rest/credentials/Credential.java b/core/src/main/java/org/apache/iceberg/rest/credentials/Credential.java new file mode 100644 index 000000000000..0bd6673384de --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/credentials/Credential.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest.credentials; + +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.immutables.value.Value; + +@Value.Immutable +public interface Credential { + String prefix(); + + Map config(); + + @Value.Check + default void validate() { + Preconditions.checkArgument(!prefix().isEmpty(), "Invalid prefix: must be non-empty"); + Preconditions.checkArgument(!config().isEmpty(), "Invalid config: must be non-empty"); + } +} diff --git a/core/src/main/java/org/apache/iceberg/rest/credentials/CredentialParser.java b/core/src/main/java/org/apache/iceberg/rest/credentials/CredentialParser.java new file mode 100644 index 000000000000..14314d6e4fb2 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/credentials/CredentialParser.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest.credentials; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import java.io.IOException; +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.JsonUtil; + +public class CredentialParser { + private static final String PREFIX = "prefix"; + private static final String CONFIG = "config"; + + private CredentialParser() {} + + public static String toJson(Credential credential) { + return toJson(credential, false); + } + + public static String toJson(Credential credential, boolean pretty) { + return JsonUtil.generate(gen -> toJson(credential, gen), pretty); + } + + public static void toJson(Credential credential, JsonGenerator gen) throws IOException { + Preconditions.checkArgument(null != credential, "Invalid credential: null"); + + gen.writeStartObject(); + + gen.writeStringField(PREFIX, credential.prefix()); + JsonUtil.writeStringMap(CONFIG, credential.config(), gen); + + gen.writeEndObject(); + } + + public static Credential fromJson(String json) { + return JsonUtil.parse(json, CredentialParser::fromJson); + } + + public static Credential fromJson(JsonNode json) { + Preconditions.checkArgument(null != json, "Cannot parse credential from null object"); + String prefix = JsonUtil.getString(PREFIX, json); + Map config = JsonUtil.getStringMap(CONFIG, json); + return ImmutableCredential.builder().prefix(prefix).config(config).build(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/ConfigResponse.java b/core/src/main/java/org/apache/iceberg/rest/responses/ConfigResponse.java index f4efc0ff281a..da22ca287b30 100644 --- a/core/src/main/java/org/apache/iceberg/rest/responses/ConfigResponse.java +++ b/core/src/main/java/org/apache/iceberg/rest/responses/ConfigResponse.java @@ -18,12 +18,16 @@ */ package org.apache.iceberg.rest.responses; +import java.util.List; import java.util.Map; import java.util.Objects; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.rest.Endpoint; import org.apache.iceberg.rest.RESTResponse; /** @@ -39,20 +43,24 @@ *

    *
  • defaults - properties that should be used as default configuration *
  • overrides - properties that should be used to override client configuration + *
  • endpoints - a list of endpoints that the server supports *
*/ public class ConfigResponse implements RESTResponse { private Map defaults; private Map overrides; + private List endpoints; public ConfigResponse() { // Required for Jackson deserialization } - private ConfigResponse(Map defaults, Map overrides) { + private ConfigResponse( + Map defaults, Map overrides, List endpoints) { this.defaults = defaults; this.overrides = overrides; + this.endpoints = endpoints; validate(); } @@ -80,6 +88,15 @@ public Map overrides() { return overrides != null ? overrides : ImmutableMap.of(); } + /** + * The list of available endpoints that the server supports + * + * @return A list of available endpoints that the server supports + */ + public List endpoints() { + return null != endpoints ? endpoints : ImmutableList.of(); + } + /** * Merge client-provided config with server side provided configuration to return a single * properties map which will be used for instantiating and configuring the REST catalog. @@ -107,6 +124,7 @@ public String toString() { return MoreObjects.toStringHelper(this) .add("defaults", defaults) .add("overrides", overrides) + .add("endpoints", endpoints) .toString(); } @@ -117,10 +135,12 @@ public static Builder builder() { public static class Builder { private final Map defaults; private final Map overrides; + private final List endpoints; private Builder() { this.defaults = Maps.newHashMap(); this.overrides = Maps.newHashMap(); + this.endpoints = Lists.newArrayList(); } public Builder withDefault(String key, String value) { @@ -153,8 +173,13 @@ public Builder withOverrides(Map overridesToAdd) { return this; } + public Builder withEndpoints(List endpointsToAdd) { + endpoints.addAll(endpointsToAdd); + return this; + } + public ConfigResponse build() { - return new ConfigResponse(defaults, overrides); + return new ConfigResponse(defaults, overrides, endpoints); } } } diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/ConfigResponseParser.java b/core/src/main/java/org/apache/iceberg/rest/responses/ConfigResponseParser.java index 3240840e3e93..acadcce6d4bf 100644 --- a/core/src/main/java/org/apache/iceberg/rest/responses/ConfigResponseParser.java +++ b/core/src/main/java/org/apache/iceberg/rest/responses/ConfigResponseParser.java @@ -21,13 +21,16 @@ import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.databind.JsonNode; import java.io.IOException; +import java.util.stream.Collectors; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.rest.Endpoint; import org.apache.iceberg.util.JsonUtil; public class ConfigResponseParser { private static final String DEFAULTS = "defaults"; private static final String OVERRIDES = "overrides"; + private static final String ENDPOINTS = "endpoints"; private ConfigResponseParser() {} @@ -46,6 +49,12 @@ public static void toJson(ConfigResponse response, JsonGenerator gen) throws IOE JsonUtil.writeStringMap(DEFAULTS, response.defaults(), gen); JsonUtil.writeStringMap(OVERRIDES, response.overrides(), gen); + if (!response.endpoints().isEmpty()) { + JsonUtil.writeStringArray( + ENDPOINTS, + response.endpoints().stream().map(Endpoint::toString).collect(Collectors.toList()), + gen); + } gen.writeEndObject(); } @@ -67,6 +76,13 @@ public static ConfigResponse fromJson(JsonNode json) { builder.withOverrides(JsonUtil.getStringMapNullableValues(OVERRIDES, json)); } + if (json.hasNonNull(ENDPOINTS)) { + builder.withEndpoints( + JsonUtil.getStringList(ENDPOINTS, json).stream() + .map(Endpoint::fromString) + .collect(Collectors.toList())); + } + return builder.build(); } } diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/Range.java b/core/src/main/java/org/apache/iceberg/rest/responses/LoadCredentialsResponse.java similarity index 68% rename from aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/Range.java rename to core/src/main/java/org/apache/iceberg/rest/responses/LoadCredentialsResponse.java index ff66e5c2a1cb..410981291046 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/Range.java +++ b/core/src/main/java/org/apache/iceberg/rest/responses/LoadCredentialsResponse.java @@ -16,28 +16,19 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.aliyun.oss.mock; +package org.apache.iceberg.rest.responses; -public class Range { +import java.util.List; +import org.apache.iceberg.rest.RESTResponse; +import org.apache.iceberg.rest.credentials.Credential; +import org.immutables.value.Value; - private final long start; - private final long end; - - public Range(long start, long end) { - this.start = start; - this.end = end; - } - - public long start() { - return start; - } - - public long end() { - return end; - } +@Value.Immutable +public interface LoadCredentialsResponse extends RESTResponse { + List credentials(); @Override - public String toString() { - return String.format("%d-%d", start, end); + default void validate() { + // nothing to validate } } diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/LoadCredentialsResponseParser.java b/core/src/main/java/org/apache/iceberg/rest/responses/LoadCredentialsResponseParser.java new file mode 100644 index 000000000000..9ee0b9c35e1e --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/responses/LoadCredentialsResponseParser.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest.responses; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import java.io.IOException; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.rest.credentials.Credential; +import org.apache.iceberg.rest.credentials.CredentialParser; +import org.apache.iceberg.util.JsonUtil; + +public class LoadCredentialsResponseParser { + private static final String STORAGE_CREDENTIALS = "storage-credentials"; + + private LoadCredentialsResponseParser() {} + + public static String toJson(LoadCredentialsResponse response) { + return toJson(response, false); + } + + public static String toJson(LoadCredentialsResponse response, boolean pretty) { + return JsonUtil.generate(gen -> toJson(response, gen), pretty); + } + + public static void toJson(LoadCredentialsResponse response, JsonGenerator gen) + throws IOException { + Preconditions.checkArgument(null != response, "Invalid load credentials response: null"); + + gen.writeStartObject(); + + gen.writeArrayFieldStart(STORAGE_CREDENTIALS); + for (Credential credential : response.credentials()) { + CredentialParser.toJson(credential, gen); + } + + gen.writeEndArray(); + + gen.writeEndObject(); + } + + public static LoadCredentialsResponse fromJson(String json) { + return JsonUtil.parse(json, LoadCredentialsResponseParser::fromJson); + } + + public static LoadCredentialsResponse fromJson(JsonNode json) { + Preconditions.checkArgument( + null != json, "Cannot parse load credentials response from null object"); + + JsonNode credentials = JsonUtil.get(STORAGE_CREDENTIALS, json); + Preconditions.checkArgument( + credentials.isArray(), "Cannot parse credentials from non-array: %s", credentials); + + ImmutableLoadCredentialsResponse.Builder builder = ImmutableLoadCredentialsResponse.builder(); + for (JsonNode credential : credentials) { + builder.addCredentials(CredentialParser.fromJson(credential)); + } + + return builder.build(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java b/core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java index 5e5353c3909a..977220e7d782 100644 --- a/core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java +++ b/core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java @@ -18,13 +18,17 @@ */ package org.apache.iceberg.rest.responses; +import java.util.List; import java.util.Map; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.rest.RESTResponse; +import org.apache.iceberg.rest.credentials.Credential; /** * A REST response that is used when a table is successfully loaded. @@ -39,16 +43,22 @@ public class LoadTableResponse implements RESTResponse { private String metadataLocation; private TableMetadata metadata; private Map config; + private TableMetadata metadataWithLocation; + private List credentials; public LoadTableResponse() { // Required for Jackson deserialization } private LoadTableResponse( - String metadataLocation, TableMetadata metadata, Map config) { + String metadataLocation, + TableMetadata metadata, + Map config, + List credentials) { this.metadataLocation = metadataLocation; this.metadata = metadata; this.config = config; + this.credentials = credentials; } @Override @@ -61,13 +71,22 @@ public String metadataLocation() { } public TableMetadata tableMetadata() { - return TableMetadata.buildFrom(metadata).withMetadataLocation(metadataLocation).build(); + if (null == metadataWithLocation) { + this.metadataWithLocation = + TableMetadata.buildFrom(metadata).withMetadataLocation(metadataLocation).build(); + } + + return metadataWithLocation; } public Map config() { return config != null ? config : ImmutableMap.of(); } + public List credentials() { + return credentials != null ? credentials : ImmutableList.of(); + } + @Override public String toString() { return MoreObjects.toStringHelper(this) @@ -85,6 +104,7 @@ public static class Builder { private String metadataLocation; private TableMetadata metadata; private final Map config = Maps.newHashMap(); + private final List credentials = Lists.newArrayList(); private Builder() {} @@ -104,9 +124,19 @@ public Builder addAllConfig(Map properties) { return this; } + public Builder addCredential(Credential credential) { + credentials.add(credential); + return this; + } + + public Builder addAllCredentials(List credentialsToAdd) { + credentials.addAll(credentialsToAdd); + return this; + } + public LoadTableResponse build() { Preconditions.checkNotNull(metadata, "Invalid metadata: null"); - return new LoadTableResponse(metadataLocation, metadata, config); + return new LoadTableResponse(metadataLocation, metadata, config, credentials); } } } diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponseParser.java b/core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponseParser.java new file mode 100644 index 000000000000..8d34b1498369 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponseParser.java @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest.responses; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import java.io.IOException; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableMetadataParser; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.rest.credentials.Credential; +import org.apache.iceberg.rest.credentials.CredentialParser; +import org.apache.iceberg.util.JsonUtil; + +public class LoadTableResponseParser { + + private static final String METADATA_LOCATION = "metadata-location"; + private static final String METADATA = "metadata"; + private static final String CONFIG = "config"; + private static final String STORAGE_CREDENTIALS = "storage-credentials"; + + private LoadTableResponseParser() {} + + public static String toJson(LoadTableResponse response) { + return toJson(response, false); + } + + public static String toJson(LoadTableResponse response, boolean pretty) { + return JsonUtil.generate(gen -> toJson(response, gen), pretty); + } + + public static void toJson(LoadTableResponse response, JsonGenerator gen) throws IOException { + Preconditions.checkArgument(null != response, "Invalid load table response: null"); + + gen.writeStartObject(); + + if (null != response.metadataLocation()) { + gen.writeStringField(METADATA_LOCATION, response.metadataLocation()); + } + + gen.writeFieldName(METADATA); + TableMetadataParser.toJson(response.tableMetadata(), gen); + + if (!response.config().isEmpty()) { + JsonUtil.writeStringMap(CONFIG, response.config(), gen); + } + + if (!response.credentials().isEmpty()) { + gen.writeArrayFieldStart(STORAGE_CREDENTIALS); + for (Credential credential : response.credentials()) { + CredentialParser.toJson(credential, gen); + } + + gen.writeEndArray(); + } + + gen.writeEndObject(); + } + + public static LoadTableResponse fromJson(String json) { + return JsonUtil.parse(json, LoadTableResponseParser::fromJson); + } + + public static LoadTableResponse fromJson(JsonNode json) { + Preconditions.checkArgument(null != json, "Cannot parse load table response from null object"); + + String metadataLocation = null; + if (json.hasNonNull(METADATA_LOCATION)) { + metadataLocation = JsonUtil.getString(METADATA_LOCATION, json); + } + + TableMetadata metadata = TableMetadataParser.fromJson(JsonUtil.get(METADATA, json)); + + if (null != metadataLocation) { + metadata = TableMetadata.buildFrom(metadata).withMetadataLocation(metadataLocation).build(); + } + + LoadTableResponse.Builder builder = LoadTableResponse.builder().withTableMetadata(metadata); + + if (json.hasNonNull(CONFIG)) { + builder.addAllConfig(JsonUtil.getStringMap(CONFIG, json)); + } + + if (json.hasNonNull(STORAGE_CREDENTIALS)) { + builder.addAllCredentials(LoadCredentialsResponseParser.fromJson(json).credentials()); + } + + return builder.build(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponse.java b/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponse.java index d07ba872fdaa..d7f9040e77f7 100644 --- a/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponse.java +++ b/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponse.java @@ -18,8 +18,11 @@ */ package org.apache.iceberg.rest.responses; +import java.util.List; import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.rest.RESTResponse; +import org.apache.iceberg.rest.credentials.Credential; import org.apache.iceberg.view.ViewMetadata; import org.immutables.value.Value; @@ -31,6 +34,11 @@ public interface LoadViewResponse extends RESTResponse { Map config(); + @Value.Default + default List credentials() { + return ImmutableList.of(); + } + @Override default void validate() { // nothing to validate as it's not possible to create an invalid instance diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponseParser.java b/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponseParser.java index a8aaf17e5d76..aedf05cf62a9 100644 --- a/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponseParser.java +++ b/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponseParser.java @@ -22,6 +22,8 @@ import com.fasterxml.jackson.databind.JsonNode; import java.io.IOException; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.rest.credentials.Credential; +import org.apache.iceberg.rest.credentials.CredentialParser; import org.apache.iceberg.util.JsonUtil; import org.apache.iceberg.view.ViewMetadata; import org.apache.iceberg.view.ViewMetadataParser; @@ -31,6 +33,7 @@ public class LoadViewResponseParser { private static final String METADATA_LOCATION = "metadata-location"; private static final String METADATA = "metadata"; private static final String CONFIG = "config"; + private static final String STORAGE_CREDENTIALS = "storage-credentials"; private LoadViewResponseParser() {} @@ -56,6 +59,15 @@ public static void toJson(LoadViewResponse response, JsonGenerator gen) throws I JsonUtil.writeStringMap(CONFIG, response.config(), gen); } + if (!response.credentials().isEmpty()) { + gen.writeArrayFieldStart(STORAGE_CREDENTIALS); + for (Credential credential : response.credentials()) { + CredentialParser.toJson(credential, gen); + } + + gen.writeEndArray(); + } + gen.writeEndObject(); } @@ -80,6 +92,10 @@ public static LoadViewResponse fromJson(JsonNode json) { builder.config(JsonUtil.getStringMap(CONFIG, json)); } + if (json.hasNonNull(STORAGE_CREDENTIALS)) { + builder.addAllCredentials(LoadCredentialsResponseParser.fromJson(json).credentials()); + } + return builder.build(); } } diff --git a/core/src/main/java/org/apache/iceberg/util/ContentFileUtil.java b/core/src/main/java/org/apache/iceberg/util/ContentFileUtil.java index fc68f14d0971..04fc077d10ea 100644 --- a/core/src/main/java/org/apache/iceberg/util/ContentFileUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/ContentFileUtil.java @@ -75,4 +75,9 @@ public static CharSequence referencedDataFile(DeleteFile deleteFile) { return null; } } + + public static String referencedDataFileLocation(DeleteFile deleteFile) { + CharSequence location = referencedDataFile(deleteFile); + return location != null ? location.toString() : null; + } } diff --git a/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java b/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java index 27cd96a39733..d40f64844797 100644 --- a/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java +++ b/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java @@ -85,17 +85,16 @@ static class ParallelIterator implements CloseableIterator { private final ExecutorService workerPool; private final CompletableFuture>>[] taskFutures; private final ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); - private final int maxQueueSize; private final AtomicBoolean closed = new AtomicBoolean(false); private ParallelIterator( Iterable> iterables, ExecutorService workerPool, int maxQueueSize) { + Preconditions.checkArgument(maxQueueSize > 0, "Max queue size must be greater than 0"); this.tasks = Iterables.transform( iterables, iterable -> new Task<>(iterable, queue, closed, maxQueueSize)) .iterator(); this.workerPool = workerPool; - this.maxQueueSize = maxQueueSize; // submit 2 tasks per worker at a time this.taskFutures = new CompletableFuture[2 * ThreadPools.WORKER_THREAD_POOL_SIZE]; } @@ -195,12 +194,9 @@ public synchronized boolean hasNext() { // If the consumer is processing records more slowly than the producers, the producers will // eventually fill the queue and yield, returning continuations. Continuations and new tasks // are started by checkTasks(). The check here prevents us from restarting continuations or - // starting new tasks too early (when queue is almost full) or too late (when queue is already - // emptied). Restarting too early would lead to tasks yielding very quickly (CPU waste on - // scheduling). Restarting too late would mean the consumer may need to wait for the tasks - // to produce new items. A consumer slower than producers shouldn't need to wait. - int queueLowWaterMark = maxQueueSize / 2; - if (queue.size() > queueLowWaterMark) { + // starting new tasks before the queue is emptied. Restarting too early would lead to tasks + // yielding very quickly (CPU waste on scheduling). + if (!queue.isEmpty()) { return true; } diff --git a/core/src/main/java/org/apache/iceberg/util/ThreadPools.java b/core/src/main/java/org/apache/iceberg/util/ThreadPools.java index ced121c03c63..0d7d174a10f5 100644 --- a/core/src/main/java/org/apache/iceberg/util/ThreadPools.java +++ b/core/src/main/java/org/apache/iceberg/util/ThreadPools.java @@ -42,13 +42,14 @@ private ThreadPools() {} public static final int WORKER_THREAD_POOL_SIZE = SystemConfigs.WORKER_THREAD_POOL_SIZE.value(); - private static final ExecutorService WORKER_POOL = newWorkerPool("iceberg-worker-pool"); + private static final ExecutorService WORKER_POOL = + newExitingWorkerPool("iceberg-worker-pool", WORKER_THREAD_POOL_SIZE); public static final int DELETE_WORKER_THREAD_POOL_SIZE = SystemConfigs.DELETE_WORKER_THREAD_POOL_SIZE.value(); private static final ExecutorService DELETE_WORKER_POOL = - newWorkerPool("iceberg-delete-worker-pool", DELETE_WORKER_THREAD_POOL_SIZE); + newExitingWorkerPool("iceberg-delete-worker-pool", DELETE_WORKER_THREAD_POOL_SIZE); /** * Return an {@link ExecutorService} that uses the "worker" thread-pool. @@ -81,14 +82,61 @@ public static ExecutorService getDeleteWorkerPool() { return DELETE_WORKER_POOL; } + /** + * Creates a fixed-size thread pool that uses daemon threads. The pool is wrapped with {@link + * MoreExecutors#getExitingExecutorService(ThreadPoolExecutor)}, which registers a shutdown hook + * to ensure the pool terminates when the JVM exits. Important: Even if the pool is + * explicitly shut down using {@link ExecutorService#shutdown()}, the shutdown hook is not + * removed. This can lead to accumulation of shutdown hooks if this method is used repeatedly for + * short-lived thread pools. + * + *

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

For clarity and to avoid potential issues with shutdown hook accumulation, prefer using + * either {@link #newExitingWorkerPool(String, int)} or {@link #newFixedThreadPool(String, int)}, + * depending on the intended lifecycle of the thread pool. + * + * @deprecated will be removed in 2.0.0. Use {@link #newExitingWorkerPool(String, int)} for + * long-lived thread pools that require a shutdown hook, or {@link #newFixedThreadPool(String, + * int)} for short-lived thread pools where you manage the lifecycle. + */ + @Deprecated public static ExecutorService newWorkerPool(String namePrefix, int poolSize) { + return newExitingWorkerPool(namePrefix, poolSize); + } + + /** + * Creates a fixed-size thread pool that uses daemon threads and registers a shutdown hook to + * ensure the pool terminates when the JVM exits. This is suitable for long-lived thread pools + * that should be automatically cleaned up on JVM shutdown. + */ + public static ExecutorService newExitingWorkerPool(String namePrefix, int poolSize) { return MoreExecutors.getExitingExecutorService( - (ThreadPoolExecutor) - Executors.newFixedThreadPool(poolSize, newDaemonThreadFactory(namePrefix))); + (ThreadPoolExecutor) newFixedThreadPool(namePrefix, poolSize)); + } + + /** Creates a fixed-size thread pool that uses daemon threads. */ + public static ExecutorService newFixedThreadPool(String namePrefix, int poolSize) { + return Executors.newFixedThreadPool(poolSize, newDaemonThreadFactory(namePrefix)); } /** diff --git a/core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java b/core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java index 4a5805742443..7edca59731a5 100644 --- a/core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java +++ b/core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java @@ -50,41 +50,45 @@ static ByteBuffer allocatePrimitiveBuffer() { return ByteBuffer.allocate(PRIMITIVE_BUFFER_SIZE); } - /** - * Signed ints do not have their bytes in magnitude order because of the sign bit. To fix this, - * flip the sign bit so that all negatives are ordered before positives. This essentially shifts - * the 0 value so that we don't break our ordering when we cross the new 0 value. - */ + /** Internally just calls {@link #wholeNumberOrderedBytes(long, ByteBuffer)} */ public static ByteBuffer intToOrderedBytes(int val, ByteBuffer reuse) { - ByteBuffer bytes = ByteBuffers.reuse(reuse, PRIMITIVE_BUFFER_SIZE); - bytes.putLong(((long) val) ^ 0x8000000000000000L); - return bytes; + return wholeNumberOrderedBytes(val, reuse); + } + + /** Internally just calls {@link #wholeNumberOrderedBytes(long, ByteBuffer)} */ + public static ByteBuffer longToOrderedBytes(long val, ByteBuffer reuse) { + return wholeNumberOrderedBytes(val, reuse); + } + + /** Internally just calls {@link #wholeNumberOrderedBytes(long, ByteBuffer)} */ + public static ByteBuffer shortToOrderedBytes(short val, ByteBuffer reuse) { + return wholeNumberOrderedBytes(val, reuse); + } + + /** Internally just calls {@link #wholeNumberOrderedBytes(long, ByteBuffer)} */ + public static ByteBuffer tinyintToOrderedBytes(byte val, ByteBuffer reuse) { + return wholeNumberOrderedBytes(val, reuse); } /** - * Signed longs are treated the same as the signed ints in {@link #intToOrderedBytes(int, - * ByteBuffer)} + * Signed longs do not have their bytes in magnitude order because of the sign bit. To fix this, + * flip the sign bit so that all negatives are ordered before positives. This essentially shifts + * the 0 value so that we don't break our ordering when we cross the new 0 value. */ - public static ByteBuffer longToOrderedBytes(long val, ByteBuffer reuse) { + public static ByteBuffer wholeNumberOrderedBytes(long val, ByteBuffer reuse) { ByteBuffer bytes = ByteBuffers.reuse(reuse, PRIMITIVE_BUFFER_SIZE); bytes.putLong(val ^ 0x8000000000000000L); return bytes; } - /** - * Signed shorts are treated the same as the signed ints in {@link #intToOrderedBytes(int, - * ByteBuffer)} - */ - public static ByteBuffer shortToOrderedBytes(short val, ByteBuffer reuse) { - return intToOrderedBytes(val, reuse); + /** Internally just calls {@link #floatingPointOrderedBytes(double, ByteBuffer)} */ + public static ByteBuffer floatToOrderedBytes(float val, ByteBuffer reuse) { + return floatingPointOrderedBytes(val, reuse); } - /** - * Signed tiny ints are treated the same as the signed ints in {@link #intToOrderedBytes(int, - * ByteBuffer)} - */ - public static ByteBuffer tinyintToOrderedBytes(byte val, ByteBuffer reuse) { - return intToOrderedBytes(val, reuse); + /** Internally just calls {@link #floatingPointOrderedBytes(double, ByteBuffer)} */ + public static ByteBuffer doubleToOrderedBytes(double val, ByteBuffer reuse) { + return floatingPointOrderedBytes(val, reuse); } /** @@ -92,15 +96,10 @@ public static ByteBuffer tinyintToOrderedBytes(byte val, ByteBuffer reuse) { * y), they are ordered the same way when their bits are reinterpreted as sign-magnitude * integers.” * - *

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

Which means doubles can be treated as sign magnitude integers which can then be converted * into lexicographically comparable bytes */ - public static ByteBuffer floatToOrderedBytes(float val, ByteBuffer reuse) { - return doubleToOrderedBytes(val, reuse); - } - - /** Doubles are treated the same as floats in {@link #floatToOrderedBytes(float, ByteBuffer)} */ - public static ByteBuffer doubleToOrderedBytes(double val, ByteBuffer reuse) { + public static ByteBuffer floatingPointOrderedBytes(double val, ByteBuffer reuse) { ByteBuffer bytes = ByteBuffers.reuse(reuse, PRIMITIVE_BUFFER_SIZE); long lval = Double.doubleToLongBits(val); lval ^= ((lval >> (Integer.SIZE - 1)) | Long.MIN_VALUE); diff --git a/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java b/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java index df96b90eb728..d78aeea6aae5 100644 --- a/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java +++ b/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.view; +import java.util.Locale; import java.util.UUID; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; @@ -157,7 +158,8 @@ private String newMetadataFilePath(ViewMetadata metadata, int newVersion) { ViewProperties.METADATA_COMPRESSION, ViewProperties.METADATA_COMPRESSION_DEFAULT); String fileExtension = TableMetadataParser.getFileExtension(codecName); return metadataFileLocation( - metadata, String.format("%05d-%s%s", newVersion, UUID.randomUUID(), fileExtension)); + metadata, + String.format(Locale.ROOT, "%05d-%s%s", newVersion, UUID.randomUUID(), fileExtension)); } private String metadataFileLocation(ViewMetadata metadata, String filename) { diff --git a/core/src/main/java/org/apache/iceberg/view/ViewVersionParser.java b/core/src/main/java/org/apache/iceberg/view/ViewVersionParser.java index 2645e40d94ef..06ee3b2648d2 100644 --- a/core/src/main/java/org/apache/iceberg/view/ViewVersionParser.java +++ b/core/src/main/java/org/apache/iceberg/view/ViewVersionParser.java @@ -84,12 +84,14 @@ public static ViewVersion fromJson(JsonNode node) { long timestamp = JsonUtil.getLong(TIMESTAMP_MS, node); Map summary = JsonUtil.getStringMap(SUMMARY, node); - JsonNode serializedRepresentations = node.get(REPRESENTATIONS); + JsonNode serializedRepresentations = JsonUtil.get(REPRESENTATIONS, node); + Preconditions.checkArgument( + serializedRepresentations.isArray(), + "Cannot parse representations from non-array: %s", + serializedRepresentations); ImmutableList.Builder representations = ImmutableList.builder(); for (JsonNode serializedRepresentation : serializedRepresentations) { - ViewRepresentation representation = - ViewRepresentationParser.fromJson(serializedRepresentation); - representations.add(representation); + representations.add(ViewRepresentationParser.fromJson(serializedRepresentation)); } String defaultCatalog = JsonUtil.getStringOrNull(DEFAULT_CATALOG, node); diff --git a/core/src/test/java/org/apache/iceberg/DataTableScanTestBase.java b/core/src/test/java/org/apache/iceberg/DataTableScanTestBase.java index 04bbcf662225..8bd7a5ea70d9 100644 --- a/core/src/test/java/org/apache/iceberg/DataTableScanTestBase.java +++ b/core/src/test/java/org/apache/iceberg/DataTableScanTestBase.java @@ -23,12 +23,15 @@ import static org.assertj.core.api.Assumptions.assumeThat; import java.io.IOException; +import java.util.Collection; import java.util.List; import java.util.UUID; +import java.util.stream.Collectors; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.CharSequenceMap; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -180,12 +183,25 @@ public void testSettingInvalidRefFails() { private void validateExpectedFileScanTasks(ScanT scan, List expectedFileScanPaths) throws IOException { + validateExpectedFileScanTasks(scan, expectedFileScanPaths, null); + } + + private void validateExpectedFileScanTasks( + ScanT scan, + Collection expectedFileScanPaths, + CharSequenceMap fileToManifest) + throws IOException { try (CloseableIterable scanTasks = scan.planFiles()) { assertThat(scanTasks).hasSameSizeAs(expectedFileScanPaths); List actualFiles = Lists.newArrayList(); for (T task : scanTasks) { - actualFiles.add(((FileScanTask) task).file().path()); + DataFile dataFile = ((FileScanTask) task).file(); + actualFiles.add(dataFile.path()); + if (fileToManifest != null) { + assertThat(fileToManifest.get(dataFile.path())).isEqualTo(dataFile.manifestLocation()); + } } + assertThat(actualFiles).containsAll(expectedFileScanPaths); } } @@ -246,4 +262,51 @@ public void testSequenceNumbersThroughPlanFiles() { .isEqualTo(expectedDeleteSequenceNumber); } } + + @TestTemplate + public void testManifestLocationsInScan() throws IOException { + table.newFastAppend().appendFile(FILE_A).commit(); + ManifestFile firstDataManifest = table.currentSnapshot().allManifests(table.io()).get(0); + table.newFastAppend().appendFile(FILE_B).appendFile(FILE_C).commit(); + ManifestFile secondDataManifest = + table.currentSnapshot().dataManifests(table.io()).stream() + .filter(manifest -> manifest.snapshotId() == table.currentSnapshot().snapshotId()) + .collect(Collectors.toList()) + .get(0); + CharSequenceMap fileToManifest = CharSequenceMap.create(); + fileToManifest.put(FILE_A.path(), firstDataManifest.path()); + fileToManifest.put(FILE_B.path(), secondDataManifest.path()); + fileToManifest.put(FILE_C.path(), secondDataManifest.path()); + + validateExpectedFileScanTasks(newScan(), fileToManifest.keySet(), fileToManifest); + } + + @TestTemplate + public void testManifestLocationsInScanWithDeleteFiles() throws IOException { + assumeThat(formatVersion).isEqualTo(2); + + table.newFastAppend().appendFile(FILE_A).commit(); + ManifestFile firstManifest = table.currentSnapshot().allManifests(table.io()).get(0); + DeleteFile deleteFile = newDeleteFile("data_bucket=0"); + table.newRowDelta().addDeletes(deleteFile).commit(); + CharSequenceMap fileToManifest = CharSequenceMap.create(); + fileToManifest.put(FILE_A.path(), firstManifest.path()); + ScanT scan = newScan(); + validateExpectedFileScanTasks(scan, ImmutableList.of(FILE_A.path()), fileToManifest); + List deletes = Lists.newArrayList(); + try (CloseableIterable scanTasks = scan.planFiles()) { + for (T task : scanTasks) { + FileScanTask fileScanTask = (FileScanTask) task; + deletes.addAll(fileScanTask.deletes()); + } + } + + assertThat(deletes.size()).isEqualTo(1); + ManifestFile deleteManifest = + table.currentSnapshot().deleteManifests(table.io()).stream() + .filter(manifest -> manifest.snapshotId() == table.currentSnapshot().snapshotId()) + .collect(Collectors.toList()) + .get(0); + assertThat(deletes.get(0).manifestLocation()).isEqualTo(deleteManifest.path()); + } } diff --git a/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java b/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java index 836a1ddd80f5..986e8608c082 100644 --- a/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java +++ b/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java @@ -139,6 +139,8 @@ public void testMinSequenceNumberFilteringForFiles() { DataFile file = unpartitionedFile(partSpec); + assertThat(index.hasEqualityDeletes()).isTrue(); + assertThat(index.hasPositionDeletes()).isFalse(); assertThat(index.forDataFile(0, file)).as("Only one delete file should apply").hasSize(1); } @@ -158,6 +160,9 @@ public void testUnpartitionedDeletes() { .specsById(ImmutableMap.of(partSpec.specId(), partSpec, 1, SPEC)) .build(); + assertThat(index.hasEqualityDeletes()).isTrue(); + assertThat(index.hasPositionDeletes()).isTrue(); + DataFile unpartitionedFile = unpartitionedFile(partSpec); assertThat(index.forDataFile(0, unpartitionedFile)) .as("All deletes should apply to seq 0") @@ -213,6 +218,9 @@ public void testPartitionedDeleteIndex() { .specsById(ImmutableMap.of(SPEC.specId(), SPEC, 1, PartitionSpec.unpartitioned())) .build(); + assertThat(index.hasEqualityDeletes()).isTrue(); + assertThat(index.hasPositionDeletes()).isTrue(); + assertThat(index.forDataFile(0, FILE_A)) .as("All deletes should apply to seq 0") .isEqualTo(deleteFiles); diff --git a/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java b/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java index 98a6eafaf8f6..f66496ae6624 100644 --- a/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java +++ b/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java @@ -19,27 +19,48 @@ package org.apache.iceberg; import java.nio.ByteBuffer; +import java.util.Comparator; import java.util.Map; import java.util.Random; import java.util.UUID; import java.util.concurrent.ThreadLocalRandom; +import org.apache.iceberg.MetricsModes.Counts; +import org.apache.iceberg.MetricsModes.MetricsMode; +import org.apache.iceberg.MetricsModes.None; +import org.apache.iceberg.MetricsModes.Truncate; import org.apache.iceberg.io.DeleteSchemaUtil; import org.apache.iceberg.io.LocationProvider; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.transforms.Transform; +import org.apache.iceberg.transforms.Transforms; +import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Type.PrimitiveType; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; +import org.apache.iceberg.util.RandomUtil; public class FileGenerationUtil { private FileGenerationUtil() {} public static DataFile generateDataFile(Table table, StructLike partition) { + return generateDataFile(table, partition, ImmutableMap.of(), ImmutableMap.of()); + } + + public static DataFile generateDataFile( + Table table, + StructLike partition, + Map lowerBounds, + Map upperBounds) { Schema schema = table.schema(); PartitionSpec spec = table.spec(); LocationProvider locations = table.locationProvider(); String path = locations.newDataLocation(spec, partition, generateFileName()); long fileSize = generateFileSize(); - Metrics metrics = generateRandomMetrics(schema); + MetricsConfig metricsConfig = MetricsConfig.forTable(table); + Metrics metrics = generateRandomMetrics(schema, metricsConfig, lowerBounds, upperBounds); return DataFiles.builder(spec) .withPath(path) .withPartition(partition) @@ -65,6 +86,21 @@ public static DeleteFile generatePositionDeleteFile(Table table, StructLike part .build(); } + public static DeleteFile generateEqualityDeleteFile(Table table, StructLike partition) { + PartitionSpec spec = table.spec(); + LocationProvider locations = table.locationProvider(); + String path = locations.newDataLocation(spec, partition, generateFileName()); + long fileSize = generateFileSize(); + return FileMetadata.deleteFileBuilder(spec) + .ofEqualityDeletes() + .withPartition(partition) + .withPath(path) + .withFileSizeInBytes(fileSize) + .withFormat(FileFormat.PARQUET) + .withRecordCount(generateRowCount()) + .build(); + } + public static DeleteFile generatePositionDeleteFile(Table table, DataFile dataFile) { PartitionSpec spec = table.spec(); StructLike partition = dataFile.partition(); @@ -91,7 +127,11 @@ public static String generateFileName() { return String.format("%d-%d-%s-%d.parquet", partitionId, taskId, operationId, fileCount); } - public static Metrics generateRandomMetrics(Schema schema) { + public static Metrics generateRandomMetrics( + Schema schema, + MetricsConfig metricsConfig, + Map knownLowerBounds, + Map knownUpperBounds) { long rowCount = generateRowCount(); Map columnSizes = Maps.newHashMap(); Map valueCounts = Maps.newHashMap(); @@ -106,12 +146,16 @@ public static Metrics generateRandomMetrics(Schema schema) { valueCounts.put(fieldId, generateValueCount()); nullValueCounts.put(fieldId, (long) random().nextInt(5)); nanValueCounts.put(fieldId, (long) random().nextInt(5)); - byte[] lower = new byte[16]; - random().nextBytes(lower); - lowerBounds.put(fieldId, ByteBuffer.wrap(lower)); - byte[] upper = new byte[16]; - random().nextBytes(upper); - upperBounds.put(fieldId, ByteBuffer.wrap(upper)); + if (knownLowerBounds.containsKey(fieldId) && knownUpperBounds.containsKey(fieldId)) { + lowerBounds.put(fieldId, knownLowerBounds.get(fieldId)); + upperBounds.put(fieldId, knownUpperBounds.get(fieldId)); + } else if (column.type().isPrimitiveType()) { + PrimitiveType type = column.type().asPrimitiveType(); + MetricsMode metricsMode = metricsConfig.columnMode(column.name()); + Pair bounds = generateBounds(type, metricsMode); + lowerBounds.put(fieldId, bounds.first()); + upperBounds.put(fieldId, bounds.second()); + } } return new Metrics( @@ -185,6 +229,37 @@ private static long generateFileSize() { return random().nextInt(50_000); } + private static Pair generateBounds(PrimitiveType type, MetricsMode mode) { + Comparator cmp = Comparators.forType(type); + Object value1 = generateBound(type, mode); + Object value2 = generateBound(type, mode); + if (cmp.compare(value1, value2) > 0) { + ByteBuffer lowerBuffer = Conversions.toByteBuffer(type, value2); + ByteBuffer upperBuffer = Conversions.toByteBuffer(type, value1); + return Pair.of(lowerBuffer, upperBuffer); + } else { + ByteBuffer lowerBuffer = Conversions.toByteBuffer(type, value1); + ByteBuffer upperBuffer = Conversions.toByteBuffer(type, value2); + return Pair.of(lowerBuffer, upperBuffer); + } + } + + private static Object generateBound(PrimitiveType type, MetricsMode mode) { + if (mode instanceof None || mode instanceof Counts) { + return null; + } else if (mode instanceof Truncate) { + Object value = RandomUtil.generatePrimitive(type, random()); + Transform truncate = Transforms.truncate(((Truncate) mode).length()); + if (truncate.canTransform(type)) { + return truncate.bind(type).apply(value); + } else { + return value; + } + } else { + return RandomUtil.generatePrimitive(type, random()); + } + } + private static Random random() { return ThreadLocalRandom.current(); } diff --git a/core/src/test/java/org/apache/iceberg/TestBase.java b/core/src/test/java/org/apache/iceberg/TestBase.java index e03a1efd5156..f3bbb7979547 100644 --- a/core/src/test/java/org/apache/iceberg/TestBase.java +++ b/core/src/test/java/org/apache/iceberg/TestBase.java @@ -27,10 +27,12 @@ import java.nio.file.Path; import java.nio.file.Paths; import java.util.Arrays; +import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.Set; import java.util.UUID; +import org.apache.iceberg.avro.AvroSchemaUtil; import org.apache.iceberg.deletes.PositionDelete; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.OutputFile; @@ -349,7 +351,24 @@ > ManifestEntry manifestEntry( Long fileSequenceNumber, F file) { - GenericManifestEntry entry = new GenericManifestEntry<>(table.spec().partitionType()); + Schema manifestEntrySchema; + switch (table.ops().current().formatVersion()) { + case 1: + manifestEntrySchema = V1Metadata.entrySchema(table.spec().partitionType()); + break; + case 2: + manifestEntrySchema = V2Metadata.entrySchema(table.spec().partitionType()); + break; + case 3: + manifestEntrySchema = V3Metadata.entrySchema(table.spec().partitionType()); + break; + default: + throw new IllegalArgumentException( + "Unsupported format version: " + table.ops().current().formatVersion()); + } + + GenericManifestEntry entry = + new GenericManifestEntry<>(AvroSchemaUtil.convert(manifestEntrySchema, "manifest_entry")); switch (status) { case ADDED: if (dataSequenceNumber != null && dataSequenceNumber != 0) { @@ -460,6 +479,10 @@ void validateSnapshot(Snapshot old, Snapshot snap, Long sequenceNumber, DataFile } void validateTableFiles(Table tbl, DataFile... expectedFiles) { + validateTableFiles(tbl, Arrays.asList(expectedFiles)); + } + + void validateTableFiles(Table tbl, Collection expectedFiles) { Set expectedFilePaths = Sets.newHashSet(); for (DataFile file : expectedFiles) { expectedFilePaths.add(file.path()); @@ -642,6 +665,10 @@ protected DeleteFile newEqualityDeleteFile(int specId, String partitionPath, int .build(); } + protected PositionDelete positionDelete(CharSequence path, long pos) { + return positionDelete(path, pos, null /* no row */); + } + protected PositionDelete positionDelete(CharSequence path, long pos, T row) { PositionDelete positionDelete = PositionDelete.create(); return positionDelete.set(path, pos, row); diff --git a/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java b/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java index 18e3de240170..4928f998f3b1 100644 --- a/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java +++ b/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java @@ -412,7 +412,20 @@ public void testDeleteValidateFileExistence() { assertThatThrownBy( () -> commit(table, table.newDelete().deleteFile(FILE_B).validateFilesExist(), branch)) - .isInstanceOf(ValidationException.class); + .isInstanceOf(ValidationException.class) + .hasMessage("Missing required files to delete: /path/to/data-b.parquet"); + + assertThatThrownBy( + () -> + commit( + table, + table + .newDelete() + .deleteFile("/path/to/non-existing.parquet") + .validateFilesExist(), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessage("Missing required files to delete: /path/to/non-existing.parquet"); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestFastAppend.java b/core/src/test/java/org/apache/iceberg/TestFastAppend.java index 8125c528d9c3..8da9cb0e33ef 100644 --- a/core/src/test/java/org/apache/iceberg/TestFastAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestFastAppend.java @@ -42,6 +42,25 @@ protected static List parameters() { return Arrays.asList(1, 2, 3); } + @TestTemplate + public void testAddManyFiles() { + assertThat(listManifestFiles()).as("Table should start empty").isEmpty(); + + List dataFiles = Lists.newArrayList(); + + for (int ordinal = 0; ordinal < 2 * SnapshotProducer.MIN_FILE_GROUP_SIZE; ordinal++) { + StructLike partition = TestHelpers.Row.of(ordinal % 2); + DataFile dataFile = FileGenerationUtil.generateDataFile(table, partition); + dataFiles.add(dataFile); + } + + AppendFiles append = table.newFastAppend(); + dataFiles.forEach(append::appendFile); + append.commit(); + + validateTableFiles(table, dataFiles); + } + @TestTemplate public void appendNullFile() { assertThatThrownBy(() -> table.newFastAppend().appendFile(null).commit()) @@ -252,6 +271,31 @@ public void testFailure() { assertThat(new File(newManifest.path())).doesNotExist(); } + @TestTemplate + public void testIncreaseNumRetries() { + TestTables.TestTableOperations ops = table.ops(); + ops.failCommits(TableProperties.COMMIT_NUM_RETRIES_DEFAULT + 1); + + AppendFiles append = table.newFastAppend().appendFile(FILE_B); + + // Default number of retries results in a failed commit + assertThatThrownBy(append::commit) + .isInstanceOf(CommitFailedException.class) + .hasMessage("Injected failure"); + + // After increasing the number of retries the commit succeeds + table + .updateProperties() + .set( + TableProperties.COMMIT_NUM_RETRIES, + String.valueOf(TableProperties.COMMIT_NUM_RETRIES_DEFAULT + 1)) + .commit(); + + append.commit(); + + validateSnapshot(null, readMetadata().currentSnapshot(), FILE_B); + } + @TestTemplate public void testAppendManifestCleanup() throws IOException { // inject 5 failures diff --git a/core/src/test/java/org/apache/iceberg/TestFileGenerationUtil.java b/core/src/test/java/org/apache/iceberg/TestFileGenerationUtil.java new file mode 100644 index 000000000000..ea44aa73c6d6 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestFileGenerationUtil.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; + +import java.nio.ByteBuffer; +import java.util.Comparator; +import org.apache.iceberg.MetricsModes.MetricsMode; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Comparators; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Type.PrimitiveType; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.types.Types.NestedField; +import org.junit.jupiter.api.Test; + +public class TestFileGenerationUtil { + + public static final Schema SCHEMA = + new Schema( + required(1, "int_col", Types.IntegerType.get()), + required(2, "long_col", Types.LongType.get()), + required(3, "decimal_col", Types.DecimalType.of(10, 10)), + required(4, "date_col", Types.DateType.get()), + required(5, "timestamp_col", Types.TimestampType.withoutZone()), + required(6, "timestamp_tz_col", Types.TimestampType.withZone()), + required(7, "str_col", Types.StringType.get())); + + @Test + public void testBoundsWithDefaultMetricsConfig() { + MetricsConfig metricsConfig = MetricsConfig.getDefault(); + Metrics metrics = + FileGenerationUtil.generateRandomMetrics( + SCHEMA, + metricsConfig, + ImmutableMap.of() /* no known lower bounds */, + ImmutableMap.of() /* no known upper bounds */); + + assertThat(metrics.lowerBounds()).hasSize(SCHEMA.columns().size()); + assertThat(metrics.upperBounds()).hasSize(SCHEMA.columns().size()); + + checkBounds(metrics, metricsConfig); + } + + @Test + public void testBoundsWithSpecificValues() { + MetricsConfig metricsConfig = MetricsConfig.getDefault(); + NestedField intField = SCHEMA.findField("int_col"); + PrimitiveType type = intField.type().asPrimitiveType(); + ByteBuffer intLower = Conversions.toByteBuffer(type, 0); + ByteBuffer intUpper = Conversions.toByteBuffer(type, Integer.MAX_VALUE); + Metrics metrics = + FileGenerationUtil.generateRandomMetrics( + SCHEMA, + metricsConfig, + ImmutableMap.of(intField.fieldId(), intLower), + ImmutableMap.of(intField.fieldId(), intUpper)); + + assertThat(metrics.lowerBounds()).hasSize(SCHEMA.columns().size()); + assertThat(metrics.upperBounds()).hasSize(SCHEMA.columns().size()); + + checkBounds(metrics, metricsConfig); + + ByteBuffer actualIntLower = metrics.lowerBounds().get(intField.fieldId()); + ByteBuffer actualIntUpper = metrics.upperBounds().get(intField.fieldId()); + assertThat(actualIntLower).isEqualTo(intLower); + assertThat(actualIntUpper).isEqualTo(intUpper); + } + + private void checkBounds(Metrics metrics, MetricsConfig metricsConfig) { + for (NestedField field : SCHEMA.columns()) { + MetricsMode mode = metricsConfig.columnMode(field.name()); + ByteBuffer lowerBuffer = metrics.lowerBounds().get(field.fieldId()); + ByteBuffer upperBuffer = metrics.upperBounds().get(field.fieldId()); + if (mode.equals(MetricsModes.None.get()) || mode.equals(MetricsModes.Counts.get())) { + assertThat(lowerBuffer).isNull(); + assertThat(upperBuffer).isNull(); + } else { + checkBounds(field.type().asPrimitiveType(), lowerBuffer, upperBuffer); + } + } + } + + private void checkBounds(PrimitiveType type, ByteBuffer lowerBuffer, ByteBuffer upperBuffer) { + Object lower = Conversions.fromByteBuffer(type, lowerBuffer); + Object upper = Conversions.fromByteBuffer(type, upperBuffer); + Comparator cmp = Comparators.forType(type); + assertThat(cmp.compare(lower, upper)).isLessThanOrEqualTo(0); + } +} diff --git a/core/src/test/java/org/apache/iceberg/TestLocationProvider.java b/core/src/test/java/org/apache/iceberg/TestLocationProvider.java index 7afb69483490..7edba51c3d85 100644 --- a/core/src/test/java/org/apache/iceberg/TestLocationProvider.java +++ b/core/src/test/java/org/apache/iceberg/TestLocationProvider.java @@ -240,7 +240,6 @@ public void testObjectStorageLocationProviderPathResolution() { String dataPath = "s3://random/data/location"; table.updateProperties().set(TableProperties.WRITE_DATA_LOCATION, dataPath).commit(); - assertThat(table.locationProvider().newDataLocation("file")) .as("write data path should be used when set") .contains(dataPath); @@ -279,12 +278,12 @@ public void testObjectStorageWithinTableLocation() { String fileLocation = table.locationProvider().newDataLocation("test.parquet"); String relativeLocation = fileLocation.replaceFirst(table.location(), ""); List parts = Splitter.on("/").splitToList(relativeLocation); - - assertThat(parts).hasSize(4); + assertThat(parts).hasSize(7); assertThat(parts).first().asString().isEmpty(); assertThat(parts).element(1).asString().isEqualTo("data"); - assertThat(parts).element(2).asString().isNotEmpty(); - assertThat(parts).element(3).asString().isEqualTo("test.parquet"); + // entropy dirs in the middle + assertThat(parts).elements(2, 3, 4, 5).asString().isNotEmpty(); + assertThat(parts).element(6).asString().isEqualTo("test.parquet"); } @TestTemplate @@ -304,4 +303,35 @@ public void testEncodedFieldNameInPartitionPath() { assertThat(partitionString).isEqualTo("data%231=val%231"); } + + @TestTemplate + public void testExcludePartitionInPath() { + // Update the table to use a string field for partitioning with special characters in the name + table.updateProperties().set(TableProperties.OBJECT_STORE_ENABLED, "true").commit(); + table + .updateProperties() + .set(TableProperties.WRITE_OBJECT_STORE_PARTITIONED_PATHS, "false") + .commit(); + + // Use a partition value that has a special character + StructLike partitionData = TestHelpers.CustomRow.of(0, "val"); + String fileLocation = + table.locationProvider().newDataLocation(table.spec(), partitionData, "test.parquet"); + + // no partition values included in the path and last part of entropy is seperated with "-" + assertThat(fileLocation).endsWith("/data/0110/1010/0011/11101000-test.parquet"); + } + + @TestTemplate + public void testHashInjection() { + table.updateProperties().set(TableProperties.OBJECT_STORE_ENABLED, "true").commit(); + assertThat(table.locationProvider().newDataLocation("a")) + .endsWith("/data/0101/0110/1001/10110010/a"); + assertThat(table.locationProvider().newDataLocation("b")) + .endsWith("/data/1110/0111/1110/00000011/b"); + assertThat(table.locationProvider().newDataLocation("c")) + .endsWith("/data/0010/1101/0110/01011111/c"); + assertThat(table.locationProvider().newDataLocation("d")) + .endsWith("/data/1001/0001/0100/01110011/d"); + } } diff --git a/core/src/test/java/org/apache/iceberg/TestManifestReader.java b/core/src/test/java/org/apache/iceberg/TestManifestReader.java index 999fd0d92eb2..e45415f1f2d2 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestReader.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestReader.java @@ -40,7 +40,12 @@ public class TestManifestReader extends TestBase { private static final RecursiveComparisonConfiguration FILE_COMPARISON_CONFIG = RecursiveComparisonConfiguration.builder() .withIgnoredFields( - "dataSequenceNumber", "fileOrdinal", "fileSequenceNumber", "fromProjectionPos") + "dataSequenceNumber", + "fileOrdinal", + "fileSequenceNumber", + "fromProjectionPos", + "manifestLocation", + "partitionData.partitionType.fieldsById") .build(); @TestTemplate @@ -133,6 +138,16 @@ public void testDataFilePositions() throws IOException { } } + @TestTemplate + public void testDataFileManifestPaths() throws IOException { + ManifestFile manifest = writeManifest(1000L, FILE_A, FILE_B, FILE_C); + try (ManifestReader reader = ManifestFiles.read(manifest, FILE_IO)) { + for (DataFile file : reader) { + assertThat(file.manifestLocation()).isEqualTo(manifest.path()); + } + } + } + @TestTemplate public void testDeleteFilePositions() throws IOException { assumeThat(formatVersion).as("Delete files only work for format version 2").isEqualTo(2); @@ -151,6 +166,21 @@ public void testDeleteFilePositions() throws IOException { } } + @TestTemplate + public void testDeleteFileManifestPaths() throws IOException { + assumeThat(formatVersion) + .as("Delete files only work for format version 2 or higher") + .isGreaterThanOrEqualTo(2); + ManifestFile manifest = + writeDeleteManifest(formatVersion, 1000L, FILE_A_DELETES, FILE_B_DELETES); + try (ManifestReader reader = + ManifestFiles.readDeleteManifest(manifest, FILE_IO, null)) { + for (DeleteFile file : reader) { + assertThat(file.manifestLocation()).isEqualTo(manifest.path()); + } + } + } + @TestTemplate public void testDataFileSplitOffsetsNullWhenInvalid() throws IOException { DataFile invalidOffset = diff --git a/core/src/test/java/org/apache/iceberg/TestMergeAppend.java b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java index abfcb318334b..e079f634014a 100644 --- a/core/src/test/java/org/apache/iceberg/TestMergeAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java @@ -33,9 +33,11 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import org.apache.iceberg.ManifestEntry.Status; +import org.apache.iceberg.TestHelpers.Row; import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; import org.junit.jupiter.api.TestTemplate; @@ -62,6 +64,25 @@ public void appendNullFile() { .hasMessage("Invalid data file: null"); } + @TestTemplate + public void testAddManyFiles() { + assertThat(listManifestFiles()).as("Table should start empty").isEmpty(); + + List dataFiles = Lists.newArrayList(); + + for (int ordinal = 0; ordinal < 2 * SnapshotProducer.MIN_FILE_GROUP_SIZE; ordinal++) { + StructLike partition = Row.of(ordinal % 2); + DataFile dataFile = FileGenerationUtil.generateDataFile(table, partition); + dataFiles.add(dataFile); + } + + AppendFiles append = table.newAppend(); + dataFiles.forEach(append::appendFile); + append.commit(); + + validateTableFiles(table, dataFiles); + } + @TestTemplate public void testEmptyTableAppend() { assertThat(listManifestFiles()).isEmpty(); diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java index bf6456e85aec..f8c34019875f 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java @@ -169,7 +169,9 @@ private String partitionColumn(String colName) { } } - /** @return a basic expression that always evaluates to true, to test AND logic */ + /** + * @return a basic expression that always evaluates to true, to test AND logic + */ private Expression dummyExpression() { switch (type) { case FILES: diff --git a/core/src/test/java/org/apache/iceberg/TestMetricsTruncation.java b/core/src/test/java/org/apache/iceberg/TestMetricsTruncation.java index 5bd1fcd69120..6faf1eeb5bfd 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetricsTruncation.java +++ b/core/src/test/java/org/apache/iceberg/TestMetricsTruncation.java @@ -202,11 +202,20 @@ public void testTruncateStringMax() { String test5 = "\uDBFF\uDFFF\uDBFF\uDFFF"; String test6 = "\uD800\uDFFF\uD800\uDFFF"; // Increment the previous character - String test6_2_expected = "\uD801\uDC00"; + String test6_1_expected = "\uD801\uDC00"; String test7 = "\uD83D\uDE02\uD83D\uDE02\uD83D\uDE02"; String test7_2_expected = "\uD83D\uDE02\uD83D\uDE03"; String test7_1_expected = "\uD83D\uDE03"; + // Increment the max UTF-8 character will overflow + String test8 = "a\uDBFF\uDFFFc"; + String test8_2_expected = "b"; + + // Increment skip invalid Unicode scalar values [Character.MIN_SURROGATE, + // Character.MAX_SURROGATE] + String test9 = "a" + (char) (Character.MIN_SURROGATE - 1) + "b"; + String test9_2_expected = "a" + (char) (Character.MAX_SURROGATE + 1); + Comparator cmp = Literal.of(test1).comparator(); assertThat(cmp.compare(truncateStringMax(Literal.of(test1), 4).value(), test1)) .as("Truncated upper bound should be greater than or equal to the actual upper bound") @@ -254,10 +263,10 @@ public void testTruncateStringMax() { assertThat(truncateStringMax(Literal.of(test5), 1)) .as("An upper bound doesn't exist since the first two characters are max UTF-8 characters") .isNull(); - assertThat(cmp.compare(truncateStringMax(Literal.of(test6), 2).value(), test6)) + assertThat(cmp.compare(truncateStringMax(Literal.of(test6), 1).value(), test6)) .as("Truncated upper bound should be greater than or equal to the actual upper bound") .isGreaterThanOrEqualTo(0); - assertThat(cmp.compare(truncateStringMax(Literal.of(test6), 1).value(), test6_2_expected)) + assertThat(cmp.compare(truncateStringMax(Literal.of(test6), 1).value(), test6_1_expected)) .as( "Test 4 byte UTF-8 character increment. Output must have one character with " + "the first character incremented") @@ -273,5 +282,24 @@ public void testTruncateStringMax() { .as( "Test input with multiple 4 byte UTF-8 character where the first unicode character should be incremented") .isEqualTo(0); + + assertThat(cmp.compare(truncateStringMax(Literal.of(test8), 2).value(), test8)) + .as("Truncated upper bound should be greater than or equal to the actual upper bound") + .isGreaterThanOrEqualTo(0); + assertThat(cmp.compare(truncateStringMax(Literal.of(test8), 2).value(), test8_2_expected)) + .as( + "Test the last character is the 4-byte max UTF-8 character after truncated where the second-to-last " + + "character should be incremented") + .isEqualTo(0); + + assertThat(cmp.compare(truncateStringMax(Literal.of(test9), 2).value(), test9)) + .as("Truncated upper bound should be greater than or equal to the actual upper bound") + .isGreaterThanOrEqualTo(0); + + assertThat(cmp.compare(truncateStringMax(Literal.of(test9), 2).value(), test9_2_expected)) + .as( + "Test the last character is `Character.MIN_SURROGATE - 1` after truncated, it should be incremented to " + + "next valid Unicode scalar value `Character.MAX_SURROGATE + 1`") + .isEqualTo(0); } } diff --git a/core/src/test/java/org/apache/iceberg/TestPartitionSpecBuilderCaseSensitivity.java b/core/src/test/java/org/apache/iceberg/TestPartitionSpecBuilderCaseSensitivity.java new file mode 100644 index 000000000000..349912bd78df --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestPartitionSpecBuilderCaseSensitivity.java @@ -0,0 +1,873 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatIllegalArgumentException; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.types.Types.NestedField; +import org.apache.iceberg.types.Types.StructType; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestPartitionSpecBuilderCaseSensitivity { + + private static final int V2_FORMAT_VERSION = 2; + private static final Schema SCHEMA_WITHOUT_NAME_CONFLICTS = + new Schema( + required(1, "id", Types.IntegerType.get()), + required(2, "data", Types.StringType.get()), + required(3, "category", Types.StringType.get()), + required(4, "order_date", Types.DateType.get()), + required(5, "order_time", Types.TimestampType.withoutZone()), + required(6, "ship_date", Types.DateType.get()), + required(7, "ship_time", Types.TimestampType.withoutZone())); + + private static final Schema SCHEMA_WITH_NAME_CONFLICTS = + new Schema( + required(1, "id", Types.IntegerType.get()), + required(2, "data", Types.StringType.get()), + required(3, "DATA", Types.StringType.get()), + required(4, "order_date", Types.DateType.get()), + required(5, "ORDER_DATE", Types.DateType.get()), + required(6, "order_time", Types.TimestampType.withoutZone()), + required(7, "ORDER_TIME", Types.TimestampType.withoutZone())); + + @TempDir private Path temp; + private File tableDir = null; + + @BeforeEach + public void setupTableDir() throws IOException { + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); + } + + @AfterEach + public void cleanupTables() { + TestTables.clearTables(); + } + + @Test + public void testPartitionTypeWithColumnNamesThatDifferOnlyInLetterCase() { + Schema schema = + new Schema( + required(1, "id", Types.IntegerType.get()), + required(2, "data", Types.StringType.get()), + required(3, "DATA", Types.StringType.get()), + required(4, "order_date", Types.DateType.get())); + PartitionSpec spec = PartitionSpec.builderFor(schema).identity("data").identity("DATA").build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "data", Types.StringType.get()), + NestedField.optional(1001, "DATA", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testPartitionTypeWithIdentityTargetName() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .identity("data", "partition1") + .build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "partition1", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testBucketSourceNameAllowsExactDuplicateWhenCaseSensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .bucket("data", 10, "partition1") + .bucket("data", 10, "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.IntegerType.get()), + NestedField.optional(1001, "PARTITION1", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testBucketTargetNameDefaultValue() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS).bucket("data", 10).build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "data_bucket", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testBucketTargetNameDefaultValueCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .bucket("DATA", 10) + .build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "data_bucket", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testBucketSourceNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .bucket("data", 10, "partition1") + .bucket("DATA", 10, "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.IntegerType.get()), + NestedField.optional(1001, "PARTITION1", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testBucketTargetNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .bucket("data", 10, "partition1") + .bucket("category", 10, "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.IntegerType.get()), + NestedField.optional(1001, "PARTITION1", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testBucketTargetNameDoesNotAllowExactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .bucket("data", 10, "partition1") + .bucket("category", 10, "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testBucketTargetNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .bucket("data", 10, "partition1") + .bucket("DATA", 10, "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testTruncateTargetNameDefaultValue() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS).truncate("data", 10).build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "data_trunc", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testTruncateTargetNameDefaultValueCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .truncate("DATA", 10) + .build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "data_trunc", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testTruncateSourceNameAllowsExactDuplicateWhenCaseSensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .truncate("data", 10, "partition1") + .truncate("data", 10, "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.StringType.get()), + NestedField.optional(1001, "PARTITION1", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testTruncateSourceNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .truncate("data", 10, "partition1") + .truncate("DATA", 10, "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.StringType.get()), + NestedField.optional(1001, "PARTITION1", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testTruncateTargetNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .truncate("data", 10, "partition1") + .truncate("category", 10, "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.StringType.get()), + NestedField.optional(1001, "PARTITION1", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testTruncateTargetNameDoesNotAllowExactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .truncate("data", 10, "partition1") + .truncate("category", 10, "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testTruncateTargetNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .truncate("data", 10, "partition1") + .truncate("DATA", 10, "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testIdentityTargetNameDefaultValue() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS).identity("data").build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "data", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testIdentityTargetNameDefaultValueCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .identity("DATA") + .build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "data", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testIdentitySourceNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .identity("data", "partition1") + .identity("data", "PARTITION1") + .build()) + .withMessage( + "Cannot add redundant partition: 1000: partition1: identity(2) conflicts with 1001: PARTITION1: identity(2)"); + } + + @Test + public void testIdentitySourceNameDoesNotAllowInexactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .identity("data", "partition1") + .identity("DATA", "PARTITION1") + .build()) + .withMessage( + "Cannot add redundant partition: 1000: partition1: identity(2) conflicts with 1001: PARTITION1: identity(2)"); + } + + @Test + public void testIdentityTargetNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .identity("data", "partition1") + .identity("category", "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.StringType.get()), + NestedField.optional(1001, "PARTITION1", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testIdentityTargetNameDoesNotAllowExactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .identity("data", "partition1") + .identity("category", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testIdentityTargetNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .identity("data", "partition1") + .identity("DATA", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testAlwaysNullTargetNameDefaultValue() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS).alwaysNull("data").build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "data_null", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testAlwaysNullTargetNameDefaultValueCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .alwaysNull("DATA") + .build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "data_null", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testAlwaysNullSourceNameAllowsExactDuplicateWhenCaseSensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .alwaysNull("data", "partition1") + .alwaysNull("data", "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.StringType.get()), + NestedField.optional(1001, "PARTITION1", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testAlwaysNullSourceNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .alwaysNull("data", "partition1") + .alwaysNull("DATA", "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.StringType.get()), + NestedField.optional(1001, "PARTITION1", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testAlwaysNullTargetNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .alwaysNull("data", "partition1") + .alwaysNull("category", "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.StringType.get()), + NestedField.optional(1001, "PARTITION1", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testAlwaysNullTargetNameDoesNotAllowExactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .alwaysNull("data", "partition1") + .alwaysNull("category", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testAlwaysNullTargetNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .alwaysNull("data", "partition1") + .alwaysNull("DATA", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testYearTargetNameDefaultValue() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS).year("order_date").build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "order_date_year", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testYearTargetNameDefaultValueCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .year("ORDER_DATE") + .build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "order_date_year", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testYearSourceNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .year("order_date", "partition1") + .year("order_date", "PARTITION1") + .build()) + .withMessage( + "Cannot add redundant partition: 1000: partition1: year(4) conflicts with 1001: PARTITION1: year(4)"); + } + + @Test + public void testYearSourceNameDoesNotAllowInexactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .year("order_date", "partition1") + .year("ORDER_DATE", "PARTITION1") + .build()) + .withMessage( + "Cannot add redundant partition: 1000: partition1: year(4) conflicts with 1001: PARTITION1: year(4)"); + } + + @Test + public void testYearTargetNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .year("order_date", "partition1") + .year("ship_date", "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.IntegerType.get()), + NestedField.optional(1001, "PARTITION1", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testYearTargetNameDoesNotAllowExactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .year("order_date", "partition1") + .year("ship_date", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testYearTargetNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .year("order_date", "partition1") + .year("ORDER_DATE", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testMonthTargetNameDefaultValue() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS).month("order_date").build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "order_date_month", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testMonthTargetNameDefaultValueCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .month("ORDER_DATE") + .build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "order_date_month", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testMonthSourceNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .month("order_date", "partition1") + .month("order_date", "PARTITION1") + .build()) + .withMessage( + "Cannot add redundant partition: 1000: partition1: month(4) conflicts with 1001: PARTITION1: month(4)"); + } + + @Test + public void testMonthSourceNameDoesNotAllowInexactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .month("order_date", "partition1") + .month("ORDER_DATE", "PARTITION1") + .build()) + .withMessage( + "Cannot add redundant partition: 1000: partition1: month(4) conflicts with 1001: PARTITION1: month(4)"); + } + + @Test + public void testMonthTargetNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .month("order_date", "partition1") + .month("ship_date", "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.IntegerType.get()), + NestedField.optional(1001, "PARTITION1", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testMonthTargetNameDoesNotAllowExactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .month("order_date", "partition1") + .month("ship_date", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testMonthTargetNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .month("order_date", "partition1") + .month("ORDER_DATE", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testDayTargetNameDefaultValue() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS).day("order_date").build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "order_date_day", Types.DateType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testDayTargetNameDefaultValueCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .day("ORDER_DATE") + .build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "order_date_day", Types.DateType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testDaySourceNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .day("order_date", "partition1") + .day("order_date", "PARTITION1") + .build()) + .withMessage( + "Cannot add redundant partition: 1000: partition1: day(4) conflicts with 1001: PARTITION1: day(4)"); + } + + @Test + public void testDaySourceNameDoesNotAllowInexactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .day("order_date", "partition1") + .day("ORDER_DATE", "PARTITION1") + .build()) + .withMessage( + "Cannot add redundant partition: 1000: partition1: day(4) conflicts with 1001: PARTITION1: day(4)"); + } + + @Test + public void testDayTargetNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .day("order_date", "partition1") + .day("ship_date", "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.DateType.get()), + NestedField.optional(1001, "PARTITION1", Types.DateType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testDayTargetNameDoesNotAllowExactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .day("order_date", "partition1") + .day("ship_date", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testDayTargetNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .day("order_date", "partition1") + .day("ORDER_DATE", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testHourTargetNameDefaultValue() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS).hour("order_time").build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "order_time_hour", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testHourTargetNameDefaultValueCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .hour("ORDER_TIME") + .build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "order_time_hour", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testHourSourceNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .hour("order_time", "partition1") + .hour("order_time", "PARTITION1") + .build()) + .withMessage( + "Cannot add redundant partition: 1000: partition1: hour(6) conflicts with 1001: PARTITION1: hour(6)"); + } + + @Test + public void testHourSourceNameDoesNotAllowInexactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .hour("order_time", "partition1") + .hour("ORDER_TIME", "PARTITION1") + .build()) + .withMessage( + "Cannot add redundant partition: 1000: partition1: hour(5) conflicts with 1001: PARTITION1: hour(5)"); + } + + @Test + public void testHourTargetNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .hour("order_time", "partition1") + .hour("ship_time", "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.IntegerType.get()), + NestedField.optional(1001, "PARTITION1", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testHourTargetNameDoesNotAllowExactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .hour("order_time", "partition1") + .hour("ship_time", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testHourTargetNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .hour("order_time", "partition1") + .hour("ORDER_TIME", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } +} diff --git a/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java b/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java index a324b8af2e70..f5636a77be61 100644 --- a/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java +++ b/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatIllegalArgumentException; import static org.assertj.core.api.Assertions.entry; import java.io.File; @@ -95,6 +96,30 @@ public void testSpecInfoPartitionedTable() { .doesNotContainKey(Integer.MAX_VALUE); } + @TestTemplate + public void testSpecInfoPartitionedTableCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(schema).caseSensitive(false).identity("DATA").build(); + TestTables.TestTable table = TestTables.create(tableDir, "test", schema, spec, formatVersion); + + assertThat(table.spec()).isEqualTo(spec); + assertThat(table.spec().lastAssignedFieldId()).isEqualTo(spec.lastAssignedFieldId()); + assertThat(table.specs()) + .containsExactly(entry(spec.specId(), spec)) + .doesNotContainKey(Integer.MAX_VALUE); + } + + @TestTemplate + public void testSpecInfoPartitionedTableCaseSensitiveFails() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> { + PartitionSpec spec = + PartitionSpec.builderFor(schema).caseSensitive(true).identity("DATA").build(); + }) + .withMessage("Cannot find source column: DATA"); + } + @TestTemplate public void testColumnDropWithPartitionSpecEvolution() { PartitionSpec spec = PartitionSpec.builderFor(schema).identity("id").build(); diff --git a/core/src/test/java/org/apache/iceberg/TestPartitionStatsUtil.java b/core/src/test/java/org/apache/iceberg/TestPartitionStatsUtil.java new file mode 100644 index 000000000000..541fcd2ca22d --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestPartitionStatsUtil.java @@ -0,0 +1,429 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.File; +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.assertj.core.groups.Tuple; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestPartitionStatsUtil { + private static final Schema SCHEMA = + new Schema( + optional(1, "c1", Types.IntegerType.get()), + optional(2, "c2", Types.StringType.get()), + optional(3, "c3", Types.StringType.get())); + + protected static final PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA).identity("c2").identity("c3").build(); + + @TempDir public File temp; + + @Test + public void testPartitionStatsOnEmptyTable() throws Exception { + Table testTable = TestTables.create(tempDir("empty_table"), "empty_table", SCHEMA, SPEC, 2); + assertThatThrownBy( + () -> PartitionStatsUtil.computeStats(testTable, testTable.currentSnapshot())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("snapshot cannot be null"); + } + + @Test + public void testPartitionStatsOnUnPartitionedTable() throws Exception { + Table testTable = + TestTables.create( + tempDir("unpartitioned_table"), + "unpartitioned_table", + SCHEMA, + PartitionSpec.unpartitioned(), + 2); + + List files = prepareDataFiles(testTable); + AppendFiles appendFiles = testTable.newAppend(); + files.forEach(appendFiles::appendFile); + appendFiles.commit(); + + assertThatThrownBy( + () -> PartitionStatsUtil.computeStats(testTable, testTable.currentSnapshot())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("table must be partitioned"); + } + + @Test + public void testPartitionStats() throws Exception { + Table testTable = + TestTables.create( + tempDir("partition_stats_compute"), "partition_stats_compute", SCHEMA, SPEC, 2); + + List files = prepareDataFiles(testTable); + for (int i = 0; i < 3; i++) { + // insert same set of records thrice to have a new manifest files + AppendFiles appendFiles = testTable.newAppend(); + files.forEach(appendFiles::appendFile); + appendFiles.commit(); + } + + Snapshot snapshot1 = testTable.currentSnapshot(); + Types.StructType partitionType = Partitioning.partitionType(testTable); + computeAndValidatePartitionStats( + testTable, + Tuple.tuple( + partitionData(partitionType, "foo", "A"), + 0, + 3 * files.get(0).recordCount(), + 3, + 3 * files.get(0).fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + 0L, + snapshot1.timestampMillis(), + snapshot1.snapshotId()), + Tuple.tuple( + partitionData(partitionType, "foo", "B"), + 0, + 3 * files.get(1).recordCount(), + 3, + 3 * files.get(1).fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + 0L, + snapshot1.timestampMillis(), + snapshot1.snapshotId()), + Tuple.tuple( + partitionData(partitionType, "bar", "A"), + 0, + 3 * files.get(2).recordCount(), + 3, + 3 * files.get(2).fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + 0L, + snapshot1.timestampMillis(), + snapshot1.snapshotId()), + Tuple.tuple( + partitionData(partitionType, "bar", "B"), + 0, + 3 * files.get(3).recordCount(), + 3, + 3 * files.get(3).fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + 0L, + snapshot1.timestampMillis(), + snapshot1.snapshotId())); + + DeleteFile deleteFile = + FileGenerationUtil.generatePositionDeleteFile(testTable, TestHelpers.Row.of("foo", "A")); + testTable.newRowDelta().addDeletes(deleteFile).commit(); + Snapshot snapshot2 = testTable.currentSnapshot(); + + DeleteFile eqDelete = + FileGenerationUtil.generateEqualityDeleteFile(testTable, TestHelpers.Row.of("bar", "B")); + testTable.newRowDelta().addDeletes(eqDelete).commit(); + Snapshot snapshot3 = testTable.currentSnapshot(); + + computeAndValidatePartitionStats( + testTable, + Tuple.tuple( + partitionData(partitionType, "foo", "A"), + 0, + 3 * files.get(0).recordCount(), + 3, + 3 * files.get(0).fileSizeInBytes(), + deleteFile.recordCount(), // position delete file count + 1, // one position delete file + 0L, + 0, + 0L, + snapshot2.timestampMillis(), // new snapshot from pos delete commit + snapshot2.snapshotId()), + Tuple.tuple( + partitionData(partitionType, "foo", "B"), + 0, + 3 * files.get(1).recordCount(), + 3, + 3 * files.get(1).fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + 0L, + snapshot1.timestampMillis(), + snapshot1.snapshotId()), + Tuple.tuple( + partitionData(partitionType, "bar", "A"), + 0, + 3 * files.get(2).recordCount(), + 3, + 3 * files.get(2).fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + 0L, + snapshot1.timestampMillis(), + snapshot1.snapshotId()), + Tuple.tuple( + partitionData(partitionType, "bar", "B"), + 0, + 3 * files.get(3).recordCount(), + 3, + 3 * files.get(3).fileSizeInBytes(), + 0L, + 0, + eqDelete.recordCount(), + 1, // one equality delete file + 0L, + snapshot3.timestampMillis(), // new snapshot from equality delete commit + snapshot3.snapshotId())); + } + + @Test + @SuppressWarnings("MethodLength") + public void testPartitionStatsWithSchemaEvolution() throws Exception { + final PartitionSpec specBefore = PartitionSpec.builderFor(SCHEMA).identity("c2").build(); + + Table testTable = + TestTables.create( + tempDir("partition_stats_schema_evolve"), + "partition_stats_schema_evolve", + SCHEMA, + specBefore, + SortOrder.unsorted(), + 2); + + List dataFiles = prepareDataFilesOnePart(testTable); + for (int i = 0; i < 2; i++) { + AppendFiles appendFiles = testTable.newAppend(); + dataFiles.forEach(appendFiles::appendFile); + appendFiles.commit(); + } + Snapshot snapshot1 = testTable.currentSnapshot(); + Types.StructType partitionType = Partitioning.partitionType(testTable); + + computeAndValidatePartitionStats( + testTable, + Tuple.tuple( + partitionData(partitionType, "foo"), + 0, + 2 * dataFiles.get(0).recordCount(), + 2, + 2 * dataFiles.get(0).fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + 0L, + snapshot1.timestampMillis(), + snapshot1.snapshotId()), + Tuple.tuple( + partitionData(partitionType, "bar"), + 0, + 2 * dataFiles.get(1).recordCount(), + 2, + 2 * dataFiles.get(1).fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + 0L, + snapshot1.timestampMillis(), + snapshot1.snapshotId())); + + // Evolve the partition spec to include c3 + testTable.updateSpec().addField("c3").commit(); + List filesWithNewSpec = prepareDataFiles(testTable); + filesWithNewSpec.add( + FileGenerationUtil.generateDataFile(testTable, TestHelpers.Row.of("bar", null))); + partitionType = Partitioning.partitionType(testTable); + + AppendFiles appendFiles = testTable.newAppend(); + filesWithNewSpec.forEach(appendFiles::appendFile); + appendFiles.commit(); + Snapshot snapshot2 = testTable.currentSnapshot(); + + computeAndValidatePartitionStats( + testTable, + Tuple.tuple( + partitionData(partitionType, "foo", null), // unified tuple + 0, // old spec id as the record is unmodified + 2 * dataFiles.get(0).recordCount(), + 2, + 2 * dataFiles.get(0).fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + 0L, + snapshot1.timestampMillis(), + snapshot1.snapshotId()), + Tuple.tuple( + partitionData(partitionType, "bar", null), + 0, // old spec id for "bar, null" before evolution + 2 * dataFiles.get(1).recordCount(), + 2, + 2 * dataFiles.get(1).fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + 0L, + snapshot1.timestampMillis(), + snapshot1.snapshotId()), + Tuple.tuple( + partitionData(partitionType, "bar", null), + 1, // new spec id for "bar, null" after evolution + filesWithNewSpec.get(4).recordCount(), + 1, + filesWithNewSpec.get(4).fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + 0L, + snapshot2.timestampMillis(), // new snapshot + snapshot2.snapshotId()), + Tuple.tuple( + partitionData(partitionType, "foo", "A"), + 1, // new spec id + filesWithNewSpec.get(0).recordCount(), + 1, + filesWithNewSpec.get(0).fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + 0L, + snapshot2.timestampMillis(), // new snapshot + snapshot2.snapshotId()), + Tuple.tuple( + partitionData(partitionType, "foo", "B"), + 1, + filesWithNewSpec.get(1).recordCount(), + 1, + filesWithNewSpec.get(1).fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + 0L, + snapshot2.timestampMillis(), + snapshot2.snapshotId()), + Tuple.tuple( + partitionData(partitionType, "bar", "A"), + 1, + filesWithNewSpec.get(2).recordCount(), + 1, + filesWithNewSpec.get(2).fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + 0L, + snapshot2.timestampMillis(), + snapshot2.snapshotId()), + Tuple.tuple( + partitionData(partitionType, "bar", "B"), + 1, + filesWithNewSpec.get(3).recordCount(), + 1, + filesWithNewSpec.get(3).fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + 0L, + snapshot2.timestampMillis(), + snapshot2.snapshotId())); + } + + private static PartitionData partitionData(Types.StructType partitionType, String c2, String c3) { + PartitionData partitionData = new PartitionData(partitionType); + partitionData.set(0, c2); + partitionData.set(1, c3); + return partitionData; + } + + private static PartitionData partitionData(Types.StructType partitionType, String c2) { + PartitionData partitionData = new PartitionData(partitionType); + partitionData.set(0, c2); + return partitionData; + } + + private static List prepareDataFiles(Table table) { + List dataFiles = Lists.newArrayList(); + dataFiles.add(FileGenerationUtil.generateDataFile(table, TestHelpers.Row.of("foo", "A"))); + dataFiles.add(FileGenerationUtil.generateDataFile(table, TestHelpers.Row.of("foo", "B"))); + dataFiles.add(FileGenerationUtil.generateDataFile(table, TestHelpers.Row.of("bar", "A"))); + dataFiles.add(FileGenerationUtil.generateDataFile(table, TestHelpers.Row.of("bar", "B"))); + + return dataFiles; + } + + private static List prepareDataFilesOnePart(Table table) { + List dataFiles = Lists.newArrayList(); + dataFiles.add(FileGenerationUtil.generateDataFile(table, TestHelpers.Row.of("foo"))); + dataFiles.add(FileGenerationUtil.generateDataFile(table, TestHelpers.Row.of("bar"))); + + return dataFiles; + } + + private static void computeAndValidatePartitionStats(Table testTable, Tuple... expectedValues) { + // compute and commit partition stats file + Collection result = + PartitionStatsUtil.computeStats(testTable, testTable.currentSnapshot()); + + assertThat(result) + .extracting( + PartitionStats::partition, + PartitionStats::specId, + PartitionStats::dataRecordCount, + PartitionStats::dataFileCount, + PartitionStats::totalDataFileSizeInBytes, + PartitionStats::positionDeleteRecordCount, + PartitionStats::positionDeleteFileCount, + PartitionStats::equalityDeleteRecordCount, + PartitionStats::equalityDeleteFileCount, + PartitionStats::totalRecordCount, + PartitionStats::lastUpdatedAt, + PartitionStats::lastUpdatedSnapshotId) + .containsExactlyInAnyOrder(expectedValues); + } + + private File tempDir(String folderName) throws IOException { + return java.nio.file.Files.createTempDirectory(temp.toPath(), folderName).toFile(); + } +} diff --git a/core/src/test/java/org/apache/iceberg/TestPartitioning.java b/core/src/test/java/org/apache/iceberg/TestPartitioning.java index 91f0fe95c2fa..a4df125f1de2 100644 --- a/core/src/test/java/org/apache/iceberg/TestPartitioning.java +++ b/core/src/test/java/org/apache/iceberg/TestPartitioning.java @@ -122,6 +122,25 @@ public void testPartitionTypeWithRenamesInV1Table() { assertThat(actualType).isEqualTo(expectedType); } + @Test + public void testPartitionTypeWithRenamesInV1TableCaseInsensitive() { + PartitionSpec initialSpec = + PartitionSpec.builderFor(SCHEMA).caseSensitive(false).identity("DATA", "p1").build(); + TestTables.TestTable table = + TestTables.create(tableDir, "test", SCHEMA, initialSpec, V1_FORMAT_VERSION); + + table.updateSpec().addField("category").commit(); + + table.updateSpec().renameField("p1", "p2").commit(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "p2", Types.StringType.get()), + NestedField.optional(1001, "category", Types.StringType.get())); + StructType actualType = Partitioning.partitionType(table); + assertThat(actualType).isEqualTo(expectedType); + } + @Test public void testPartitionTypeWithAddingBackSamePartitionFieldInV1Table() { TestTables.TestTable table = @@ -252,6 +271,23 @@ public void testGroupingKeyTypeWithRenamesInV1Table() { assertThat(actualType).isEqualTo(expectedType); } + @Test + public void testGroupingKeyTypeWithRenamesInV1TableCaseInsensitive() { + PartitionSpec initialSpec = + PartitionSpec.builderFor(SCHEMA).caseSensitive(false).identity("DATA", "p1").build(); + TestTables.TestTable table = + TestTables.create(tableDir, "test", SCHEMA, initialSpec, V1_FORMAT_VERSION); + + table.updateSpec().addField("category").commit(); + + table.updateSpec().renameField("p1", "p2").commit(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "p2", Types.StringType.get())); + StructType actualType = Partitioning.groupingKeyType(table.schema(), table.specs().values()); + assertThat(actualType).isEqualTo(expectedType); + } + @Test public void testGroupingKeyTypeWithRenamesInV2Table() { PartitionSpec initialSpec = PartitionSpec.builderFor(SCHEMA).identity("data", "p1").build(); diff --git a/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java b/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java index 6a47a24fb494..f95fe6191e43 100644 --- a/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java +++ b/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java @@ -370,7 +370,7 @@ public void testRetainLastWithExpireById() { } // Retain last 3 snapshots, but explicitly remove the first snapshot - removeSnapshots(table).expireSnapshotId(firstSnapshotId).retainLast(3).commit(); + table.expireSnapshots().expireSnapshotId(firstSnapshotId).retainLast(3).commit(); assertThat(table.snapshots()).hasSize(2); assertThat(table.snapshot(firstSnapshotId)).isNull(); @@ -956,7 +956,8 @@ public void testWithExpiringStagedThenCherrypick() { List deletedFiles = Lists.newArrayList(); // Expire `B` commit. - removeSnapshots(table) + table + .expireSnapshots() .deleteWith(deletedFiles::add) .expireSnapshotId(snapshotB.snapshotId()) .commit(); @@ -1171,6 +1172,26 @@ public void testBranchExpiration() { assertThat(table.ops().current().ref(SnapshotRef.MAIN_BRANCH)).isNotNull(); } + @TestTemplate + public void testIncrementalCleanupFailsWhenExpiringSnapshotId() { + table.newAppend().appendFile(FILE_A).commit(); + table.newDelete().deleteFile(FILE_A).commit(); + long snapshotId = table.currentSnapshot().snapshotId(); + table.newAppend().appendFile(FILE_B).commit(); + waitUntilAfter(table.currentSnapshot().timestampMillis()); + RemoveSnapshots removeSnapshots = (RemoveSnapshots) table.expireSnapshots(); + + assertThatThrownBy( + () -> + removeSnapshots + .withIncrementalCleanup(true) + .expireSnapshotId(snapshotId) + .cleanExpiredFiles(true) + .commit()) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage("Cannot clean files incrementally when snapshot IDs are specified"); + } + @TestTemplate public void testMultipleRefsAndCleanExpiredFilesFailsForIncrementalCleanup() { table.newAppend().appendFile(FILE_A).commit(); diff --git a/core/src/test/java/org/apache/iceberg/TestRowDelta.java b/core/src/test/java/org/apache/iceberg/TestRowDelta.java index a2a043e630bb..1d67e48a2ce2 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowDelta.java +++ b/core/src/test/java/org/apache/iceberg/TestRowDelta.java @@ -1409,4 +1409,151 @@ public void testRowDeltaCaseSensitivity() { .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Found new conflicting delete files"); } + + @TestTemplate + public void testRewrittenDeleteFiles() { + DataFile dataFile = newDataFile("data_bucket=0"); + DeleteFile deleteFile = newDeleteFile(dataFile.specId(), "data_bucket=0"); + RowDelta baseRowDelta = table.newRowDelta().addRows(dataFile).addDeletes(deleteFile); + Snapshot baseSnapshot = commit(table, baseRowDelta, branch); + assertThat(baseSnapshot.operation()).isEqualTo(DataOperations.OVERWRITE); + + DeleteFile newDeleteFile = newDeleteFile(dataFile.specId(), "data_bucket=0"); + RowDelta rowDelta = + table + .newRowDelta() + .removeDeletes(deleteFile) + .addDeletes(newDeleteFile) + .validateFromSnapshot(baseSnapshot.snapshotId()); + Snapshot snapshot = commit(table, rowDelta, branch); + assertThat(snapshot.operation()).isEqualTo(DataOperations.DELETE); + + List dataManifests = snapshot.dataManifests(table.io()); + assertThat(dataManifests).hasSize(1); + validateManifest( + dataManifests.get(0), + dataSeqs(1L), + fileSeqs(1L), + ids(baseSnapshot.snapshotId()), + files(dataFile), + statuses(Status.ADDED)); + + List deleteManifests = snapshot.deleteManifests(table.io()); + assertThat(deleteManifests).hasSize(2); + validateDeleteManifest( + deleteManifests.get(0), + dataSeqs(2L), + fileSeqs(2L), + ids(snapshot.snapshotId()), + files(newDeleteFile), + statuses(Status.ADDED)); + validateDeleteManifest( + deleteManifests.get(1), + dataSeqs(1L), + fileSeqs(1L), + ids(snapshot.snapshotId()), + files(deleteFile), + statuses(Status.DELETED)); + } + + @TestTemplate + public void testConcurrentDeletesRewriteSameDeleteFile() { + DataFile dataFile = newDataFile("data_bucket=0"); + DeleteFile deleteFile = newDeleteFile(dataFile.specId(), "data_bucket=0"); + RowDelta baseRowDelta = table.newRowDelta().addRows(dataFile).addDeletes(deleteFile); + Snapshot baseSnapshot = commit(table, baseRowDelta, branch); + assertThat(baseSnapshot.operation()).isEqualTo(DataOperations.OVERWRITE); + + // commit the first DELETE operation that replaces `deleteFile` + DeleteFile newDeleteFile1 = newDeleteFile(dataFile.specId(), "data_bucket=0"); + RowDelta delete1 = + table + .newRowDelta() + .addDeletes(newDeleteFile1) + .removeDeletes(deleteFile) + .validateFromSnapshot(baseSnapshot.snapshotId()) + .validateNoConflictingDataFiles(); + Snapshot snapshot1 = commit(table, delete1, branch); + assertThat(snapshot1.operation()).isEqualTo(DataOperations.DELETE); + assertThat(snapshot1.sequenceNumber()).isEqualTo(2L); + + // commit the second DELETE operation that replaces `deleteFile` + DeleteFile newDeleteFile2 = newDeleteFile(dataFile.specId(), "data_bucket=0"); + RowDelta delete2 = + table + .newRowDelta() + .addDeletes(newDeleteFile2) + .removeDeletes(deleteFile) + .validateFromSnapshot(baseSnapshot.snapshotId()) + .validateNoConflictingDataFiles(); + Snapshot snapshot2 = commit(table, delete2, branch); + assertThat(snapshot2.operation()).isEqualTo(DataOperations.DELETE); + assertThat(snapshot2.sequenceNumber()).isEqualTo(3L); + + List dataManifests = snapshot2.dataManifests(table.io()); + assertThat(dataManifests).hasSize(1); + validateManifest( + dataManifests.get(0), + dataSeqs(1L), + fileSeqs(1L), + ids(baseSnapshot.snapshotId()), + files(dataFile), + statuses(Status.ADDED)); + + // verify both new delete files have been added + List deleteManifests = snapshot2.deleteManifests(table.io()); + assertThat(deleteManifests).hasSize(2); + validateDeleteManifest( + deleteManifests.get(0), + dataSeqs(3L), + fileSeqs(3L), + ids(snapshot2.snapshotId()), + files(newDeleteFile2), + statuses(Status.ADDED)); + validateDeleteManifest( + deleteManifests.get(1), + dataSeqs(2L), + fileSeqs(2L), + ids(snapshot1.snapshotId()), + files(newDeleteFile1), + statuses(Status.ADDED)); + } + + @TestTemplate + public void testConcurrentMergeRewriteSameDeleteFile() { + DataFile dataFile = newDataFile("data_bucket=0"); + DeleteFile deleteFile = newDeleteFile(dataFile.specId(), "data_bucket=0"); + RowDelta baseRowDelta = table.newRowDelta().addRows(dataFile).addDeletes(deleteFile); + Snapshot baseSnapshot = commit(table, baseRowDelta, branch); + assertThat(baseSnapshot.operation()).isEqualTo(DataOperations.OVERWRITE); + + // commit a DELETE operation that replaces `deleteFile` + DeleteFile newDeleteFile1 = newDeleteFile(dataFile.specId(), "data_bucket=0"); + RowDelta delete = + table + .newRowDelta() + .addDeletes(newDeleteFile1) + .removeDeletes(deleteFile) + .validateFromSnapshot(baseSnapshot.snapshotId()) + .validateNoConflictingDataFiles(); + commit(table, delete, branch); + + // attempt to commit a MERGE operation that replaces `deleteFile` + DataFile newDataFile2 = newDataFile("data_bucket=0"); + DeleteFile newDeleteFile2 = newDeleteFile(dataFile.specId(), "data_bucket=0"); + RowDelta merge = + table + .newRowDelta() + .addRows(newDataFile2) + .addDeletes(newDeleteFile2) + .removeDeletes(deleteFile) + .validateFromSnapshot(baseSnapshot.snapshotId()) + .validateNoConflictingDataFiles() + .validateNoConflictingDeleteFiles(); + + // MERGE must fail as DELETE could have deleted more positions + assertThatThrownBy(() -> commit(table, merge, branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Found new conflicting delete files that can apply"); + } } diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotProducer.java b/core/src/test/java/org/apache/iceberg/TestSnapshotProducer.java new file mode 100644 index 000000000000..52bffdf18587 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotProducer.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.junit.jupiter.api.Test; + +public class TestSnapshotProducer { + + @Test + public void testManifestFileGroupSize() { + assertManifestWriterCount( + 4 /* worker pool size */, + 100 /* file count */, + 1 /* manifest writer count */, + "Must use 1 writer if file count is small"); + + assertManifestWriterCount( + 4 /* worker pool size */, + SnapshotProducer.MIN_FILE_GROUP_SIZE /* file count */, + 1 /* manifest writer count */, + "Must use 1 writer if file count matches min group size"); + + assertManifestWriterCount( + 4 /* worker pool size */, + SnapshotProducer.MIN_FILE_GROUP_SIZE + 1 /* file count */, + 1 /* manifest writer count */, + "Must use 1 writer if file count is slightly above min group size"); + + assertManifestWriterCount( + 4 /* worker pool size */, + (int) (1.25 * SnapshotProducer.MIN_FILE_GROUP_SIZE) /* file count */, + 1 /* manifest writer count */, + "Must use 1 writer when file count is < 1.5 * min group size"); + + assertManifestWriterCount( + 4 /* worker pool size */, + (int) (1.5 * SnapshotProducer.MIN_FILE_GROUP_SIZE) /* file count */, + 2 /* manifest writer count */, + "Must use 2 writers when file count is >= 1.5 * min group size"); + + assertManifestWriterCount( + 3 /* worker pool size */, + 100 * SnapshotProducer.MIN_FILE_GROUP_SIZE /* file count */, + 3 /* manifest writer count */, + "Must limit parallelism to worker pool size when file count is large"); + + assertManifestWriterCount( + 32 /* worker pool size */, + 5 * SnapshotProducer.MIN_FILE_GROUP_SIZE /* file count */, + 5 /* manifest writer count */, + "Must limit parallelism to avoid tiny manifests"); + } + + private void assertManifestWriterCount( + int workerPoolSize, int fileCount, int expectedManifestWriterCount, String errMsg) { + int writerCount = SnapshotProducer.manifestWriterCount(workerPoolSize, fileCount); + assertThat(writerCount).withFailMessage(errMsg).isEqualTo(expectedManifestWriterCount); + } +} diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java index 809b1ba963c7..5ada35765773 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java @@ -1635,4 +1635,88 @@ public void buildReplacementKeepsSnapshotLog() throws Exception { .hasSize(2) .containsExactlyElementsOf(metadata.snapshotLog()); } + + @Test + public void testConstructV3Metadata() { + TableMetadata.newTableMetadata( + TEST_SCHEMA, + PartitionSpec.unpartitioned(), + SortOrder.unsorted(), + TEST_LOCATION, + ImmutableMap.of(), + 3); + } + + @Test + public void testV3TimestampNanoTypeSupport() { + Schema v3Schema = + new Schema( + Types.NestedField.required(3, "id", Types.LongType.get()), + Types.NestedField.required(4, "data", Types.StringType.get()), + Types.NestedField.required( + 5, + "struct", + Types.StructType.of( + Types.NestedField.optional( + 6, "ts_nanos", Types.TimestampNanoType.withZone())))); + + for (int unsupportedFormatVersion : ImmutableList.of(1, 2)) { + assertThatThrownBy( + () -> + TableMetadata.newTableMetadata( + v3Schema, + PartitionSpec.unpartitioned(), + SortOrder.unsorted(), + TEST_LOCATION, + ImmutableMap.of(), + unsupportedFormatVersion)) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Invalid type in v%s schema: struct.ts_nanos timestamptz_ns is not supported until v3", + unsupportedFormatVersion); + } + + // should be allowed in v3 + TableMetadata.newTableMetadata( + v3Schema, + PartitionSpec.unpartitioned(), + SortOrder.unsorted(), + TEST_LOCATION, + ImmutableMap.of(), + 3); + } + + @Test + public void onlyMetadataLocationIsUpdatedWithoutTimestampAndMetadataLogEntry() { + String uuid = "386b9f01-002b-4d8c-b77f-42c3fd3b7c9b"; + TableMetadata metadata = + TableMetadata.buildFromEmpty() + .assignUUID(uuid) + .setLocation("location") + .setCurrentSchema(TEST_SCHEMA, 3) + .addPartitionSpec(PartitionSpec.unpartitioned()) + .addSortOrder(SortOrder.unsorted()) + .discardChanges() + .withMetadataLocation("original-metadata-location") + .build(); + + assertThat(metadata.previousFiles()).isEmpty(); + assertThat(metadata.metadataFileLocation()).isEqualTo("original-metadata-location"); + + // this will only update the metadata location without writing a new metadata log entry or + // updating lastUpdatedMillis + TableMetadata newMetadata = + TableMetadata.buildFrom(metadata).withMetadataLocation("new-metadata-location").build(); + assertThat(newMetadata.lastUpdatedMillis()).isEqualTo(metadata.lastUpdatedMillis()); + assertThat(newMetadata.metadataFileLocation()).isEqualTo("new-metadata-location"); + assertThat(newMetadata.previousFiles()).isEmpty(); + + TableMetadata updatedMetadata = + TableMetadata.buildFrom(newMetadata) + .withMetadataLocation("updated-metadata-location") + .build(); + assertThat(updatedMetadata.lastUpdatedMillis()).isEqualTo(newMetadata.lastUpdatedMillis()); + assertThat(updatedMetadata.metadataFileLocation()).isEqualTo("updated-metadata-location"); + assertThat(updatedMetadata.previousFiles()).isEmpty(); + } } diff --git a/core/src/test/java/org/apache/iceberg/avro/TestReadDefaultValues.java b/core/src/test/java/org/apache/iceberg/avro/TestReadDefaultValues.java new file mode 100644 index 000000000000..1cfe88206b1b --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/avro/TestReadDefaultValues.java @@ -0,0 +1,166 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.avro; + +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.util.List; +import org.apache.avro.generic.GenericData.Record; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SingleValueParser; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestReadDefaultValues { + + @TempDir public Path temp; + + private static final Object[][] TYPES_WITH_DEFAULTS = + new Object[][] { + {Types.BooleanType.get(), "true"}, + {Types.IntegerType.get(), "1"}, + {Types.LongType.get(), "9999999"}, + {Types.FloatType.get(), "1.23"}, + {Types.DoubleType.get(), "123.456"}, + {Types.DateType.get(), "\"2007-12-03\""}, + {Types.TimeType.get(), "\"10:15:30\""}, + {Types.TimestampType.withoutZone(), "\"2007-12-03T10:15:30\""}, + {Types.TimestampType.withZone(), "\"2007-12-03T10:15:30+00:00\""}, + {Types.StringType.get(), "\"foo\""}, + {Types.UUIDType.get(), "\"eb26bdb1-a1d8-4aa6-990e-da940875492c\""}, + {Types.FixedType.ofLength(2), "\"111f\""}, + {Types.BinaryType.get(), "\"0000ff\""}, + {Types.DecimalType.of(9, 4), "\"123.4500\""}, + {Types.DecimalType.of(9, 0), "\"2\""}, + // Avro doesn't support negative scale + // {Types.DecimalType.of(9, -20), "\"2E+20\""}, + // Nested type defaults are not currently allowed + }; + + @Test + public void testDefaultAppliedWhenMissingColumn() throws IOException { + for (Object[] typeAndDefault : TYPES_WITH_DEFAULTS) { + Type type = (Type) typeAndDefault[0]; + String defaultValueJson = (String) typeAndDefault[1]; + Object defaultValue = SingleValueParser.fromJson(type, defaultValueJson); + + // note that this schema does not have column "defaulted" + Schema writerSchema = new Schema(required(999, "written", Types.IntegerType.get())); + + File testFile = temp.resolve("test.avro").toFile(); + testFile.delete(); + + try (FileAppender writer = + Avro.write(Files.localOutput(testFile)) + .schema(writerSchema) + .createWriterFunc(GenericAvroWriter::create) + .named("test") + .build()) { + Record record = new Record(AvroSchemaUtil.convert(writerSchema.asStruct())); + record.put(0, 1); + writer.add(record); + } + + Schema readerSchema = + new Schema( + Types.NestedField.required("written") + .withId(999) + .ofType(Types.IntegerType.get()) + .build(), + Types.NestedField.optional("defaulted") + .withId(1000) + .ofType(type) + .withInitialDefault(defaultValue) + .build()); + + Record expectedRecord = new Record(AvroSchemaUtil.convert(readerSchema.asStruct())); + expectedRecord.put(0, 1); + expectedRecord.put(1, defaultValue); + + List rows; + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)) + .project(readerSchema) + .createResolvingReader(schema -> GenericAvroReader.create(schema)) + .build()) { + rows = Lists.newArrayList(reader); + } + + AvroTestHelpers.assertEquals(readerSchema.asStruct(), expectedRecord, rows.get(0)); + } + } + + @Test + public void testDefaultDoesNotOverrideExplicitValue() throws IOException { + for (Object[] typeAndDefault : TYPES_WITH_DEFAULTS) { + Type type = (Type) typeAndDefault[0]; + String defaultValueJson = (String) typeAndDefault[1]; + Object defaultValue = SingleValueParser.fromJson(type, defaultValueJson); + + Schema readerSchema = + new Schema( + Types.NestedField.required("written_1") + .withId(999) + .ofType(Types.IntegerType.get()) + .build(), + Types.NestedField.optional("written_2") + .withId(1000) + .ofType(type) + .withInitialDefault(defaultValue) + .build()); + + // Create a record with null value for the column with default value + Record expectedRecord = new Record(AvroSchemaUtil.convert(readerSchema.asStruct())); + expectedRecord.put(0, 1); + expectedRecord.put(1, null); + + File testFile = temp.resolve("test.avro").toFile(); + testFile.delete(); + + try (FileAppender writer = + Avro.write(Files.localOutput(testFile)) + .schema(readerSchema) + .createWriterFunc(GenericAvroWriter::create) + .named("test") + .build()) { + writer.add(expectedRecord); + } + + List rows; + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)) + .project(readerSchema) + .createReaderFunc(GenericAvroReader::create) + .build()) { + rows = Lists.newArrayList(reader); + } + + // Existence of default value should not affect the read result + AvroTestHelpers.assertEquals(readerSchema.asStruct(), expectedRecord, rows.get(0)); + } + } +} diff --git a/core/src/test/java/org/apache/iceberg/deletes/TestBitmapPositionDeleteIndex.java b/core/src/test/java/org/apache/iceberg/deletes/TestBitmapPositionDeleteIndex.java new file mode 100644 index 000000000000..c8fc723deb9e --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/deletes/TestBitmapPositionDeleteIndex.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.deletes; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.Test; + +public class TestBitmapPositionDeleteIndex { + + @Test + public void testForEach() { + long pos1 = 10L; // Container 0 (high bits = 0) + long pos2 = 1L << 33; // Container 1 (high bits = 1) + long pos3 = pos2 + 1; // Container 1 (high bits = 1) + long pos4 = 2L << 33; // Container 2 (high bits = 2) + long pos5 = pos4 + 1; // Container 2 (high bits = 2) + long pos6 = 3L << 33; // Container 3 (high bits = 3) + + PositionDeleteIndex index = new BitmapPositionDeleteIndex(); + + // add in any order + index.delete(pos1); + index.delete(pos6); + index.delete(pos2); + index.delete(pos3); + index.delete(pos5); + index.delete(pos4); + + // output must be sorted in ascending order across containers + List positions = collect(index); + assertThat(positions).containsExactly(pos1, pos2, pos3, pos4, pos5, pos6); + } + + @Test + public void testForEachEmptyBitmapIndex() { + PositionDeleteIndex index = new BitmapPositionDeleteIndex(); + List positions = collect(index); + assertThat(positions).isEmpty(); + } + + @Test + public void testForEachEmptyIndex() { + PositionDeleteIndex index = PositionDeleteIndex.empty(); + List positions = collect(index); + assertThat(positions).isEmpty(); + } + + @Test + public void testMergeBitmapIndexWithNonEmpty() { + long pos1 = 10L; // Container 0 (high bits = 0) + long pos2 = 1L << 33; // Container 1 (high bits = 1) + long pos3 = pos2 + 1; // Container 1 (high bits = 1) + long pos4 = 2L << 33; // Container 2 (high bits = 2) + + BitmapPositionDeleteIndex index1 = new BitmapPositionDeleteIndex(); + index1.delete(pos2); + index1.delete(pos1); + + BitmapPositionDeleteIndex index2 = new BitmapPositionDeleteIndex(); + index2.delete(pos4); + index2.delete(pos3); + + index1.merge(index2); + + // output must be sorted in ascending order across containers + List positions = collect(index1); + assertThat(positions).containsExactly(pos1, pos2, pos3, pos4); + } + + @Test + public void testMergeBitmapIndexWithEmpty() { + long pos1 = 10L; // Container 0 (high bits = 0) + long pos2 = 1L << 33; // Container 1 (high bits = 1) + long pos3 = pos2 + 1; // Container 1 (high bits = 1) + long pos4 = 2L << 33; // Container 2 (high bits = 2) + + BitmapPositionDeleteIndex index = new BitmapPositionDeleteIndex(); + index.delete(pos2); + index.delete(pos1); + index.delete(pos3); + index.delete(pos4); + index.merge(PositionDeleteIndex.empty()); + + // output must be sorted in ascending order across containers + List positions = collect(index); + assertThat(positions).containsExactly(pos1, pos2, pos3, pos4); + } + + private List collect(PositionDeleteIndex index) { + List positions = Lists.newArrayList(); + index.forEach(positions::add); + return positions; + } +} diff --git a/core/src/test/java/org/apache/iceberg/deletes/TestPositionFilter.java b/core/src/test/java/org/apache/iceberg/deletes/TestPositionFilter.java index 8e35c8c9fc99..2a629b2dc2b3 100644 --- a/core/src/test/java/org/apache/iceberg/deletes/TestPositionFilter.java +++ b/core/src/test/java/org/apache/iceberg/deletes/TestPositionFilter.java @@ -20,20 +20,16 @@ import static org.assertj.core.api.Assertions.assertThat; -import java.io.IOException; -import java.util.Iterator; import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Predicate; import java.util.stream.Stream; import org.apache.avro.util.Utf8; import org.apache.iceberg.StructLike; import org.apache.iceberg.TestHelpers.Row; import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.CloseableIterator; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -110,155 +106,6 @@ public void testPositionMerging() { .containsExactly(0L, 3L, 3L, 9L, 16L, 19L, 19L, 22L, 22L, 56L, 63L, 70L, 91L); } - @Test - public void testPositionStreamRowFilter() { - CloseableIterable rows = - CloseableIterable.withNoopClose( - Lists.newArrayList( - Row.of(0L, "a"), - Row.of(1L, "b"), - Row.of(2L, "c"), - Row.of(3L, "d"), - Row.of(4L, "e"), - Row.of(5L, "f"), - Row.of(6L, "g"), - Row.of(7L, "h"), - Row.of(8L, "i"), - Row.of(9L, "j"))); - - CloseableIterable deletes = - CloseableIterable.withNoopClose(Lists.newArrayList(0L, 3L, 4L, 7L, 9L)); - - CloseableIterable actual = - Deletes.streamingFilter(rows, row -> row.get(0, Long.class), deletes); - - assertThat(Iterables.transform(actual, row -> row.get(0, Long.class))) - .as("Filter should produce expected rows") - .containsExactlyElementsOf(Lists.newArrayList(1L, 2L, 5L, 6L, 8L)); - } - - @Test - public void testPositionStreamRowDeleteMarker() { - CloseableIterable rows = - CloseableIterable.withNoopClose( - Lists.newArrayList( - Row.of(0L, "a", false), - Row.of(1L, "b", false), - Row.of(2L, "c", false), - Row.of(3L, "d", false), - Row.of(4L, "e", false), - Row.of(5L, "f", false), - Row.of(6L, "g", false), - Row.of(7L, "h", false), - Row.of(8L, "i", false), - Row.of(9L, "j", false))); - - CloseableIterable deletes = - CloseableIterable.withNoopClose(Lists.newArrayList(0L, 3L, 4L, 7L, 9L)); - - CloseableIterable actual = - Deletes.streamingMarker( - rows, - row -> row.get(0, Long.class), /* row to position */ - deletes, - row -> row.set(2, true) /* delete marker */); - - assertThat(Iterables.transform(actual, row -> row.get(2, Boolean.class))) - .as("Filter should produce expected rows") - .containsExactlyElementsOf( - Lists.newArrayList(true, false, false, true, true, false, false, true, false, true)); - } - - @Test - public void testPositionStreamRowFilterWithDuplicates() { - CloseableIterable rows = - CloseableIterable.withNoopClose( - Lists.newArrayList( - Row.of(0L, "a"), - Row.of(1L, "b"), - Row.of(2L, "c"), - Row.of(3L, "d"), - Row.of(4L, "e"), - Row.of(5L, "f"), - Row.of(6L, "g"), - Row.of(7L, "h"), - Row.of(8L, "i"), - Row.of(9L, "j"))); - - CloseableIterable deletes = - CloseableIterable.withNoopClose(Lists.newArrayList(0L, 0L, 0L, 3L, 4L, 7L, 7L, 9L, 9L, 9L)); - - CloseableIterable actual = - Deletes.streamingFilter(rows, row -> row.get(0, Long.class), deletes); - - assertThat(Iterables.transform(actual, row -> row.get(0, Long.class))) - .as("Filter should produce expected rows") - .containsExactlyElementsOf(Lists.newArrayList(1L, 2L, 5L, 6L, 8L)); - } - - @Test - public void testPositionStreamRowFilterWithRowGaps() { - // test the case where row position is greater than the delete position - CloseableIterable rows = - CloseableIterable.withNoopClose( - Lists.newArrayList(Row.of(2L, "c"), Row.of(3L, "d"), Row.of(5L, "f"), Row.of(6L, "g"))); - - CloseableIterable deletes = - CloseableIterable.withNoopClose(Lists.newArrayList(0L, 2L, 3L, 4L, 7L, 9L)); - - CloseableIterable actual = - Deletes.streamingFilter(rows, row -> row.get(0, Long.class), deletes); - - assertThat(Iterables.transform(actual, row -> row.get(0, Long.class))) - .as("Filter should produce expected rows") - .containsExactlyElementsOf(Lists.newArrayList(5L, 6L)); - } - - @Test - public void testCombinedPositionStreamRowFilter() { - CloseableIterable positionDeletes1 = - CloseableIterable.withNoopClose( - Lists.newArrayList( - Row.of("file_a.avro", 0L), - Row.of("file_a.avro", 3L), - Row.of("file_a.avro", 9L), - Row.of("file_b.avro", 5L), - Row.of("file_b.avro", 6L))); - - CloseableIterable positionDeletes2 = - CloseableIterable.withNoopClose( - Lists.newArrayList( - Row.of("file_a.avro", 3L), - Row.of("file_a.avro", 4L), - Row.of("file_a.avro", 7L), - Row.of("file_b.avro", 2L))); - - CloseableIterable rows = - CloseableIterable.withNoopClose( - Lists.newArrayList( - Row.of(0L, "a"), - Row.of(1L, "b"), - Row.of(2L, "c"), - Row.of(3L, "d"), - Row.of(4L, "e"), - Row.of(5L, "f"), - Row.of(6L, "g"), - Row.of(7L, "h"), - Row.of(8L, "i"), - Row.of(9L, "j"))); - - CloseableIterable actual = - Deletes.streamingFilter( - rows, - row -> row.get(0, Long.class), - Deletes.deletePositions( - "file_a.avro", ImmutableList.of(positionDeletes1, positionDeletes2))); - - assertThat(Iterables.transform(actual, row -> row.get(0, Long.class))) - .as("Filter should produce expected rows") - .containsExactlyElementsOf(Lists.newArrayList(1L, 2L, 5L, 6L, 8L)); - } - @Test public void testPositionSetRowFilter() { CloseableIterable rows = @@ -341,105 +188,4 @@ public void testCombinedPositionSetRowFilter(ExecutorService executorService) { .as("Filter should produce expected rows") .containsExactlyElementsOf(Lists.newArrayList(1L, 2L, 5L, 6L, 8L)); } - - @Test - public void testClosePositionStreamRowDeleteMarker() { - List deletes = Lists.newArrayList(1L, 2L); - - List records = - Lists.newArrayList( - Row.of(29, "a", 1L), Row.of(43, "b", 2L), Row.of(61, "c", 3L), Row.of(89, "d", 4L)); - - CheckingClosableIterable data = new CheckingClosableIterable<>(records); - CheckingClosableIterable deletePositions = new CheckingClosableIterable<>(deletes); - - CloseableIterable posDeletesIterable = - Deletes.streamingFilter(data, row -> row.get(2, Long.class), deletePositions); - - // end iterator is always wrapped with FilterIterator - CloseableIterable eqDeletesIterable = - Deletes.filterDeleted(posDeletesIterable, i -> false, new DeleteCounter()); - List result = Lists.newArrayList(eqDeletesIterable.iterator()); - - // as first two records deleted, expect only last two records - assertThat(Iterables.transform(result, row -> row.get(2, Long.class))) - .as("Filter should produce expected rows") - .containsExactlyElementsOf(Lists.newArrayList(3L, 4L)); - - assertThat(data.isClosed).isTrue(); - assertThat(deletePositions.isClosed).isTrue(); - } - - @Test - public void testDeleteMarkerFileClosed() { - - List deletes = Lists.newArrayList(1L, 2L); - - List records = - Lists.newArrayList( - Row.of(29, "a", 1L, false), - Row.of(43, "b", 2L, false), - Row.of(61, "c", 3L, false), - Row.of(89, "d", 4L, false)); - - CheckingClosableIterable data = new CheckingClosableIterable<>(records); - CheckingClosableIterable deletePositions = new CheckingClosableIterable<>(deletes); - - CloseableIterable resultIterable = - Deletes.streamingMarker( - data, row -> row.get(2, Long.class), deletePositions, row -> row.set(3, true)); - - // end iterator is always wrapped with FilterIterator - CloseableIterable eqDeletesIterable = - Deletes.filterDeleted(resultIterable, i -> false, new DeleteCounter()); - List result = Lists.newArrayList(eqDeletesIterable.iterator()); - - // as first two records deleted, expect only those two records marked - assertThat(Iterables.transform(result, row -> row.get(3, Boolean.class))) - .as("Filter should produce expected rows") - .containsExactlyElementsOf(Lists.newArrayList(true, true, false, false)); - - assertThat(data.isClosed).isTrue(); - assertThat(deletePositions.isClosed).isTrue(); - } - - private static class CheckingClosableIterable implements CloseableIterable { - AtomicBoolean isClosed = new AtomicBoolean(false); - final Iterable iterable; - - CheckingClosableIterable(Iterable iterable) { - this.iterable = iterable; - } - - public boolean isClosed() { - return isClosed.get(); - } - - @Override - public void close() throws IOException { - isClosed.set(true); - } - - @Override - public CloseableIterator iterator() { - Iterator it = iterable.iterator(); - return new CloseableIterator() { - - @Override - public boolean hasNext() { - return it.hasNext(); - } - - @Override - public E next() { - return it.next(); - } - - @Override - public void close() { - isClosed.set(true); - } - }; - } - } } diff --git a/core/src/test/java/org/apache/iceberg/deletes/TestRoaringPositionBitmap.java b/core/src/test/java/org/apache/iceberg/deletes/TestRoaringPositionBitmap.java new file mode 100644 index 000000000000..2daf0382973b --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/deletes/TestRoaringPositionBitmap.java @@ -0,0 +1,515 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.deletes; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.net.URL; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.List; +import java.util.Random; +import java.util.Set; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.relocated.com.google.common.io.Resources; +import org.apache.iceberg.util.Pair; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestRoaringPositionBitmap { + + private static final long BITMAP_SIZE = 0xFFFFFFFFL; + private static final long BITMAP_OFFSET = BITMAP_SIZE + 1L; + private static final long CONTAINER_SIZE = Character.MAX_VALUE; + private static final long CONTAINER_OFFSET = CONTAINER_SIZE + 1L; + private static final int VALIDATION_LOOKUP_COUNT = 20_000; + private static final Set SUPPORTED_OFFICIAL_EXAMPLE_FILES = + ImmutableSet.of("64map32bitvals.bin", "64mapempty.bin", "64mapspreadvals.bin"); + + @Parameters(name = "seed = {0}, validationSeed = {1}") + protected static List parameters() { + List parameters = Lists.newArrayList(); + Random random = new Random(); + long seed = random.nextLong(); + long validationSeed = random.nextLong(); + parameters.add(new Object[] {seed, validationSeed}); + return parameters; + } + + @Parameter(index = 0) + private long seed; + + @Parameter(index = 1) + private long validationSeed; + + @TestTemplate + public void testAdd() { + RoaringPositionBitmap bitmap = new RoaringPositionBitmap(); + + bitmap.set(10L); + assertThat(bitmap.contains(10L)).isTrue(); + + bitmap.set(0L); + assertThat(bitmap.contains(0L)).isTrue(); + + bitmap.set(10L); + assertThat(bitmap.contains(10L)).isTrue(); + } + + @TestTemplate + public void testAddPositionsRequiringMultipleBitmaps() { + RoaringPositionBitmap bitmap = new RoaringPositionBitmap(); + + // construct positions that differ in their high 32-bit parts (i.e. keys) + long pos1 = ((long) 0 << 32) | 10L; // key = 0, low = 10 + long pos2 = ((long) 1 << 32) | 20L; // key = 1, low = 20 + long pos3 = ((long) 2 << 32) | 30L; // key = 2, low = 30 + long pos4 = ((long) 100 << 32) | 40L; // key = 100, low = 40 + + bitmap.set(pos1); + bitmap.set(pos2); + bitmap.set(pos3); + bitmap.set(pos4); + + assertThat(bitmap.contains(pos1)).isTrue(); + assertThat(bitmap.contains(pos2)).isTrue(); + assertThat(bitmap.contains(pos3)).isTrue(); + assertThat(bitmap.contains(pos4)).isTrue(); + assertThat(bitmap.cardinality()).isEqualTo(4); + assertThat(bitmap.serializedSizeInBytes()).isGreaterThan(4); + assertThat(bitmap.allocatedBitmapCount()).isEqualTo(101 /* max key + 1 */); + } + + @TestTemplate + public void testAddRange() { + RoaringPositionBitmap bitmap = new RoaringPositionBitmap(); + + long posStartInclusive = 10L; + long posEndExclusive = 20L; + bitmap.setRange(posStartInclusive, posEndExclusive); + + // assert that all positions in the range [10, 20) are added + for (long pos = posStartInclusive; pos < posEndExclusive; pos++) { + assertThat(bitmap.contains(pos)).isTrue(); + } + + // assert that positions outside the range are not present + assertThat(bitmap.contains(9L)).isFalse(); + assertThat(bitmap.contains(20L)).isFalse(); + + // assert that the cardinality is correct (10 positions in range [10, 20)) + assertThat(bitmap.cardinality()).isEqualTo(10); + } + + @TestTemplate + public void testAddRangeAcrossKeys() { + RoaringPositionBitmap bitmap = new RoaringPositionBitmap(); + + long posStartInclusive = ((long) 1 << 32) - 5L; + long posEndExclusive = ((long) 1 << 32) + 5L; + bitmap.setRange(posStartInclusive, posEndExclusive); + + // assert that all positions in the range are added + for (long pos = posStartInclusive; pos < posEndExclusive; pos++) { + assertThat(bitmap.contains(pos)).isTrue(); + } + + // assert that positions outside the range are not present + assertThat(bitmap.contains(0)).isFalse(); + assertThat(bitmap.contains(posEndExclusive)).isFalse(); + + // assert that the cardinality is correct + assertThat(bitmap.cardinality()).isEqualTo(10); + } + + @TestTemplate + public void testAddEmptyRange() { + RoaringPositionBitmap bitmap = new RoaringPositionBitmap(); + bitmap.setRange(10, 10); + assertThat(bitmap.isEmpty()).isTrue(); + } + + @TestTemplate + public void testAddAll() { + RoaringPositionBitmap bitmap1 = new RoaringPositionBitmap(); + bitmap1.set(10L); + bitmap1.set(20L); + + RoaringPositionBitmap bitmap2 = new RoaringPositionBitmap(); + bitmap2.set(30L); + bitmap2.set(40L); + bitmap2.set((long) 2 << 32); + + bitmap1.setAll(bitmap2); + + assertThat(bitmap1.contains(10L)).isTrue(); + assertThat(bitmap1.contains(20L)).isTrue(); + assertThat(bitmap1.contains(30L)).isTrue(); + assertThat(bitmap1.contains(40L)).isTrue(); + assertThat(bitmap1.contains((long) 2 << 32)).isTrue(); + assertThat(bitmap1.cardinality()).isEqualTo(5); + + assertThat(bitmap2.contains(10L)).isFalse(); + assertThat(bitmap2.contains(20L)).isFalse(); + assertThat(bitmap2.cardinality()).isEqualTo(3); + } + + @TestTemplate + public void testAddAllWithEmptyBitmap() { + RoaringPositionBitmap bitmap1 = new RoaringPositionBitmap(); + bitmap1.set(10L); + bitmap1.set(20L); + + RoaringPositionBitmap emptyBitmap = new RoaringPositionBitmap(); + + bitmap1.setAll(emptyBitmap); + + assertThat(bitmap1.contains(10L)).isTrue(); + assertThat(bitmap1.contains(20L)).isTrue(); + assertThat(bitmap1.cardinality()).isEqualTo(2); + + assertThat(emptyBitmap.contains(10L)).isFalse(); + assertThat(emptyBitmap.contains(20L)).isFalse(); + assertThat(emptyBitmap.cardinality()).isEqualTo(0); + assertThat(emptyBitmap.isEmpty()).isTrue(); + } + + @TestTemplate + public void testAddAllWithOverlappingBitmap() { + RoaringPositionBitmap bitmap1 = new RoaringPositionBitmap(); + bitmap1.set(10L); + bitmap1.set(20L); + bitmap1.set(30L); + + RoaringPositionBitmap bitmap2 = new RoaringPositionBitmap(); + bitmap2.set(20L); + bitmap2.set(40L); + + bitmap1.setAll(bitmap2); + + assertThat(bitmap1.contains(10L)).isTrue(); + assertThat(bitmap1.contains(20L)).isTrue(); + assertThat(bitmap1.contains(30L)).isTrue(); + assertThat(bitmap1.contains(40L)).isTrue(); + assertThat(bitmap1.cardinality()).isEqualTo(4); + + assertThat(bitmap2.contains(10L)).isFalse(); + assertThat(bitmap2.contains(20L)).isTrue(); + assertThat(bitmap2.contains(30L)).isFalse(); + assertThat(bitmap2.contains(40L)).isTrue(); + assertThat(bitmap2.cardinality()).isEqualTo(2); + } + + @TestTemplate + public void testAddAllSparseBitmaps() { + RoaringPositionBitmap bitmap1 = new RoaringPositionBitmap(); + bitmap1.set((long) 0 << 32 | 100L); // key = 0, low = 100 + bitmap1.set((long) 1 << 32 | 200L); // key = 1, low = 200 + + RoaringPositionBitmap bitmap2 = new RoaringPositionBitmap(); + bitmap2.set((long) 2 << 32 | 300L); // key = 2, low = 300 + bitmap2.set((long) 3 << 32 | 400L); // key = 3, low = 400 + + bitmap1.setAll(bitmap2); + + assertThat(bitmap1.contains((long) 0 << 32 | 100L)).isTrue(); + assertThat(bitmap1.contains((long) 1 << 32 | 200L)).isTrue(); + assertThat(bitmap1.contains((long) 2 << 32 | 300L)).isTrue(); + assertThat(bitmap1.contains((long) 3 << 32 | 400L)).isTrue(); + assertThat(bitmap1.cardinality()).isEqualTo(4); + } + + @TestTemplate + public void testCardinality() { + RoaringPositionBitmap bitmap = new RoaringPositionBitmap(); + + assertThat(bitmap.cardinality()).isEqualTo(0); + + bitmap.set(10L); + bitmap.set(20L); + bitmap.set(30L); + + assertThat(bitmap.cardinality()).isEqualTo(3); + + bitmap.set(10L); // already exists + + assertThat(bitmap.cardinality()).isEqualTo(3); + } + + @TestTemplate + public void testCardinalitySparseBitmaps() { + RoaringPositionBitmap bitmap = new RoaringPositionBitmap(); + + bitmap.set((long) 0 << 32 | 100L); // key = 0, low = 100 + bitmap.set((long) 0 << 32 | 101L); // key = 0, low = 101 + bitmap.set((long) 0 << 32 | 105L); // key = 0, low = 101 + bitmap.set((long) 1 << 32 | 200L); // key = 1, low = 200 + bitmap.set((long) 100 << 32 | 300L); // key = 100, low = 300 + + assertThat(bitmap.cardinality()).isEqualTo(5); + } + + @TestTemplate + public void testSerializeDeserializeAllContainerBitmap() { + RoaringPositionBitmap bitmap = new RoaringPositionBitmap(); + + // bitmap 0, container 0 (array) + bitmap.set(position(0 /* bitmap */, 0 /* container */, 5L)); + bitmap.set(position(0 /* bitmap */, 0 /* container */, 7L)); + + // bitmap 0, container 1 (array that can be compressed) + bitmap.setRange( + position(0 /* bitmap */, 1 /* container */, 1L), + position(0 /* bitmap */, 1 /* container */, 1000L)); + + // bitmap 1, container 2 (bitset) + bitmap.setRange( + position(0 /* bitmap */, 2 /* container */, 1L), + position(0 /* bitmap */, 2 /* container */, CONTAINER_OFFSET - 1L)); + + // bitmap 1, container 0 (array) + bitmap.set(position(1 /* bitmap */, 0 /* container */, 10L)); + bitmap.set(position(1 /* bitmap */, 0 /* container */, 20L)); + + // bitmap 1, container 1 (array that can be compressed) + bitmap.setRange( + position(1 /* bitmap */, 1 /* container */, 10L), + position(1 /* bitmap */, 1 /* container */, 500L)); + + // bitmap 1, container 2 (bitset) + bitmap.setRange( + position(1 /* bitmap */, 2 /* container */, 1L), + position(1 /* bitmap */, 2 /* container */, CONTAINER_OFFSET - 1)); + + assertThat(bitmap.runLengthEncode()).as("Bitmap must be RLE encoded").isTrue(); + + RoaringPositionBitmap bitmapCopy = roundTripSerialize(bitmap); + + assertThat(bitmapCopy.cardinality()).isEqualTo(bitmap.cardinality()); + bitmapCopy.forEach(position -> assertThat(bitmap.contains(position)).isTrue()); + bitmap.forEach(position -> assertThat(bitmapCopy.contains(position)).isTrue()); + } + + @TestTemplate + public void testDeserializeSupportedRoaringExamples() throws IOException { + for (String file : SUPPORTED_OFFICIAL_EXAMPLE_FILES) { + RoaringPositionBitmap bitmap = readBitmap(file); + assertThat(bitmap).isNotNull(); + } + } + + @TestTemplate + public void testDeserializeUnsupportedRoaringExample() { + // this file contains a value that is larger than the max supported value in our impl + assertThatThrownBy(() -> readBitmap("64maphighvals.bin")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid unsigned key"); + } + + @TestTemplate + public void testUnsupportedPositions() { + RoaringPositionBitmap bitmap = new RoaringPositionBitmap(); + + assertThatThrownBy(() -> bitmap.set(-1L)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Bitmap supports positions that are >= 0 and <= %s", + RoaringPositionBitmap.MAX_POSITION); + + assertThatThrownBy(() -> bitmap.contains(-1L)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Bitmap supports positions that are >= 0 and <= %s", + RoaringPositionBitmap.MAX_POSITION); + + assertThatThrownBy(() -> bitmap.set(RoaringPositionBitmap.MAX_POSITION + 1L)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Bitmap supports positions that are >= 0 and <= %s", + RoaringPositionBitmap.MAX_POSITION); + + assertThatThrownBy(() -> bitmap.contains(RoaringPositionBitmap.MAX_POSITION + 1L)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Bitmap supports positions that are >= 0 and <= %s", + RoaringPositionBitmap.MAX_POSITION); + } + + @TestTemplate + public void testInvalidSerializationByteOrder() { + assertThatThrownBy(() -> RoaringPositionBitmap.deserialize(ByteBuffer.allocate(4))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("serialization requires little-endian byte order"); + } + + @TestTemplate + public void testRandomSparseBitmap() { + Pair> bitmapAndPositions = + generateSparseBitmap( + 0L /* min position */, + (long) 5 << 32 /* max position must not need more than 5 bitmaps */, + 100_000 /* cardinality */); + RoaringPositionBitmap bitmap = bitmapAndPositions.first(); + Set positions = bitmapAndPositions.second(); + assertEqual(bitmap, positions); + assertRandomPositions(bitmap, positions); + } + + @TestTemplate + public void testRandomDenseBitmap() { + Pair> bitmapAndPositions = generateDenseBitmap(7); + RoaringPositionBitmap bitmap = bitmapAndPositions.first(); + Set positions = bitmapAndPositions.second(); + assertEqual(bitmap, positions); + assertRandomPositions(bitmap, positions); + } + + @TestTemplate + public void testRandomMixedBitmap() { + Pair> bitmapAndPositions = + generateSparseBitmap( + (long) 3 << 32 /* min position must need at least 3 bitmaps */, + (long) 5 << 32 /* max position must not need more than 5 bitmaps */, + 100_000 /* cardinality */); + RoaringPositionBitmap bitmap = bitmapAndPositions.first(); + Set positions = bitmapAndPositions.second(); + + Pair> pair1 = generateDenseBitmap(9); + bitmap.setAll(pair1.first()); + positions.addAll(pair1.second()); + + Pair> pair2 = + generateSparseBitmap( + 0 /* min position */, + (long) 3 << 32 /* max position must not need more than 3 bitmaps */, + 25_000 /* cardinality */); + bitmap.setAll(pair2.first()); + positions.addAll(pair2.second()); + + Pair> pair3 = generateDenseBitmap(3); + bitmap.setAll(pair3.first()); + positions.addAll(pair3.second()); + + Pair> pair4 = + generateSparseBitmap( + 0 /* min position */, + (long) 1 << 32 /* max position must not need more than 1 bitmap */, + 5_000 /* cardinality */); + bitmap.setAll(pair4.first()); + positions.addAll(pair4.second()); + + assertEqual(bitmap, positions); + assertRandomPositions(bitmap, positions); + } + + private Pair> generateSparseBitmap( + long minInclusive, long maxExclusive, int size) { + Random random = new Random(seed); + RoaringPositionBitmap bitmap = new RoaringPositionBitmap(); + Set positions = Sets.newHashSet(); + + while (positions.size() < size) { + long position = nextLong(random, minInclusive, maxExclusive); + positions.add(position); + bitmap.set(position); + } + + return Pair.of(bitmap, positions); + } + + private Pair> generateDenseBitmap(int requiredBitmapCount) { + Random random = new Random(seed); + RoaringPositionBitmap bitmap = new RoaringPositionBitmap(); + Set positions = Sets.newHashSet(); + long currentPosition = 0; + + while (bitmap.allocatedBitmapCount() <= requiredBitmapCount) { + long maxRunPosition = currentPosition + nextLong(random, 1000, 2 * CONTAINER_SIZE); + for (long position = currentPosition; position <= maxRunPosition; position++) { + bitmap.set(position); + positions.add(position); + } + long shift = nextLong(random, (long) (0.1 * BITMAP_SIZE), (long) (0.25 * BITMAP_SIZE)); + currentPosition = maxRunPosition + shift; + } + + return Pair.of(bitmap, positions); + } + + private void assertRandomPositions(RoaringPositionBitmap bitmap, Set positions) { + Random random = new Random(validationSeed); + for (int ordinal = 0; ordinal < VALIDATION_LOOKUP_COUNT; ordinal++) { + long position = nextLong(random, 0, RoaringPositionBitmap.MAX_POSITION); + assertThat(bitmap.contains(position)).isEqualTo(positions.contains(position)); + } + } + + private static long nextLong(Random random, long minInclusive, long maxExclusive) { + return minInclusive + (long) (random.nextDouble() * (maxExclusive - minInclusive)); + } + + private static long position(int bitmapIndex, int containerIndex, long value) { + return bitmapIndex * BITMAP_OFFSET + containerIndex * CONTAINER_OFFSET + value; + } + + private static RoaringPositionBitmap roundTripSerialize(RoaringPositionBitmap bitmap) { + ByteBuffer buffer = ByteBuffer.allocate((int) bitmap.serializedSizeInBytes()); + buffer.order(ByteOrder.LITTLE_ENDIAN); + bitmap.serialize(buffer); + buffer.flip(); + return RoaringPositionBitmap.deserialize(buffer); + } + + private static RoaringPositionBitmap readBitmap(String resourceName) throws IOException { + byte[] bytes = readTestResource(resourceName); + ByteBuffer buffer = ByteBuffer.wrap(bytes); + buffer.order(ByteOrder.LITTLE_ENDIAN); + return RoaringPositionBitmap.deserialize(buffer); + } + + private static byte[] readTestResource(String resourceName) throws IOException { + URL resource = Resources.getResource(TestRoaringPositionBitmap.class, resourceName); + return Resources.toByteArray(resource); + } + + private static void assertEqual(RoaringPositionBitmap bitmap, Set positions) { + assertEqualContent(bitmap, positions); + + RoaringPositionBitmap bitmapCopy1 = roundTripSerialize(bitmap); + assertEqualContent(bitmapCopy1, positions); + + bitmap.runLengthEncode(); + RoaringPositionBitmap bitmapCopy2 = roundTripSerialize(bitmap); + assertEqualContent(bitmapCopy2, positions); + } + + private static void assertEqualContent(RoaringPositionBitmap bitmap, Set positions) { + assertThat(bitmap.cardinality()).isEqualTo(positions.size()); + positions.forEach(position -> assertThat(bitmap.contains(position)).isTrue()); + bitmap.forEach(position -> assertThat(positions.contains(position)).isTrue()); + } +} diff --git a/core/src/test/java/org/apache/iceberg/hadoop/HadoopFileIOTest.java b/core/src/test/java/org/apache/iceberg/hadoop/HadoopFileIOTest.java index 109c88daeaa5..0ff1d98f27dd 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/HadoopFileIOTest.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/HadoopFileIOTest.java @@ -24,6 +24,7 @@ import java.io.File; import java.io.IOException; import java.io.UncheckedIOException; +import java.nio.file.Files; import java.util.List; import java.util.Random; import java.util.UUID; @@ -36,6 +37,7 @@ import org.apache.iceberg.common.DynMethods; import org.apache.iceberg.io.BulkDeletionFailureException; import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.FileIOParser; import org.apache.iceberg.io.ResolvingFileIO; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -66,8 +68,7 @@ public void testListPrefix() { List scaleSizes = Lists.newArrayList(1, 1000, 2500); - scaleSizes - .parallelStream() + scaleSizes.parallelStream() .forEach( scale -> { Path scalePath = new Path(parent, Integer.toString(scale)); @@ -101,8 +102,7 @@ public void testDeletePrefix() { List scaleSizes = Lists.newArrayList(1, 1000, 2500); - scaleSizes - .parallelStream() + scaleSizes.parallelStream() .forEach( scale -> { Path scalePath = new Path(parent, Integer.toString(scale)); @@ -178,6 +178,53 @@ public void testResolvingFileIOLoad() { assertThat(result).isInstanceOf(HadoopFileIO.class); } + @Test + public void testJsonParserWithoutHadoopConf() throws Exception { + this.hadoopFileIO = new HadoopFileIO(); + + hadoopFileIO.initialize(ImmutableMap.of("properties-bar", "2")); + assertThat(hadoopFileIO.properties().get("properties-bar")).isEqualTo("2"); + + testJsonParser(hadoopFileIO, tempDir); + } + + @Test + public void testJsonParserWithHadoopConf() throws Exception { + this.hadoopFileIO = new HadoopFileIO(); + + Configuration hadoopConf = new Configuration(); + hadoopConf.setInt("hadoop-conf-foo", 1); + hadoopFileIO.setConf(hadoopConf); + assertThat(hadoopFileIO.conf().get("hadoop-conf-foo")).isNotNull(); + + hadoopFileIO.initialize(ImmutableMap.of("properties-bar", "2")); + assertThat(hadoopFileIO.properties().get("properties-bar")).isEqualTo("2"); + + testJsonParser(hadoopFileIO, tempDir); + } + + private static void testJsonParser(HadoopFileIO hadoopFileIO, File tempDir) throws Exception { + String json = FileIOParser.toJson(hadoopFileIO); + try (FileIO deserialized = FileIOParser.fromJson(json)) { + assertThat(deserialized).isInstanceOf(HadoopFileIO.class); + HadoopFileIO deserializedHadoopFileIO = (HadoopFileIO) deserialized; + + // properties are carried over during serialization and deserialization + assertThat(deserializedHadoopFileIO.properties()).isEqualTo(hadoopFileIO.properties()); + + // FileIOParser doesn't serialize and deserialize Hadoop configuration + // so config "foo" is not restored in deserialized object. + assertThat(deserializedHadoopFileIO.conf().get("hadoop-conf-foo")).isNull(); + + // make sure deserialized io can create input file + String inputFilePath = + Files.createTempDirectory(tempDir.toPath(), "junit").toFile().getAbsolutePath() + + "/test.parquet"; + deserializedHadoopFileIO.newInputFile( + File.createTempFile("test", "parquet", tempDir).toString()); + } + } + private List createRandomFiles(Path parent, int count) { Vector paths = new Vector<>(); random diff --git a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java index 2c928c06e52b..6477dfcd00eb 100644 --- a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java +++ b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java @@ -19,9 +19,11 @@ package org.apache.iceberg.rest; import java.io.IOException; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.function.Consumer; +import java.util.stream.Collectors; import org.apache.iceberg.BaseTable; import org.apache.iceberg.BaseTransaction; import org.apache.iceberg.Table; @@ -115,61 +117,51 @@ enum Route { SEPARATE_AUTH_TOKENS_URI( HTTPMethod.POST, "https://auth-server.com/token", null, OAuthTokenResponse.class), CONFIG(HTTPMethod.GET, "v1/config", null, ConfigResponse.class), - LIST_NAMESPACES(HTTPMethod.GET, "v1/namespaces", null, ListNamespacesResponse.class), + LIST_NAMESPACES( + HTTPMethod.GET, ResourcePaths.V1_NAMESPACES, null, ListNamespacesResponse.class), CREATE_NAMESPACE( HTTPMethod.POST, - "v1/namespaces", + ResourcePaths.V1_NAMESPACES, CreateNamespaceRequest.class, CreateNamespaceResponse.class), - LOAD_NAMESPACE(HTTPMethod.GET, "v1/namespaces/{namespace}", null, GetNamespaceResponse.class), - DROP_NAMESPACE(HTTPMethod.DELETE, "v1/namespaces/{namespace}"), + LOAD_NAMESPACE(HTTPMethod.GET, ResourcePaths.V1_NAMESPACE, null, GetNamespaceResponse.class), + DROP_NAMESPACE(HTTPMethod.DELETE, ResourcePaths.V1_NAMESPACE), UPDATE_NAMESPACE( HTTPMethod.POST, - "v1/namespaces/{namespace}/properties", + ResourcePaths.V1_NAMESPACE_PROPERTIES, UpdateNamespacePropertiesRequest.class, UpdateNamespacePropertiesResponse.class), - LIST_TABLES(HTTPMethod.GET, "v1/namespaces/{namespace}/tables", null, ListTablesResponse.class), + LIST_TABLES(HTTPMethod.GET, ResourcePaths.V1_TABLES, null, ListTablesResponse.class), CREATE_TABLE( HTTPMethod.POST, - "v1/namespaces/{namespace}/tables", + ResourcePaths.V1_TABLES, CreateTableRequest.class, LoadTableResponse.class), - LOAD_TABLE( - HTTPMethod.GET, "v1/namespaces/{namespace}/tables/{name}", null, LoadTableResponse.class), + LOAD_TABLE(HTTPMethod.GET, ResourcePaths.V1_TABLE, null, LoadTableResponse.class), REGISTER_TABLE( HTTPMethod.POST, - "v1/namespaces/{namespace}/register", + ResourcePaths.V1_TABLE_REGISTER, RegisterTableRequest.class, LoadTableResponse.class), UPDATE_TABLE( - HTTPMethod.POST, - "v1/namespaces/{namespace}/tables/{name}", - UpdateTableRequest.class, - LoadTableResponse.class), - DROP_TABLE(HTTPMethod.DELETE, "v1/namespaces/{namespace}/tables/{name}"), - RENAME_TABLE(HTTPMethod.POST, "v1/tables/rename", RenameTableRequest.class, null), + HTTPMethod.POST, ResourcePaths.V1_TABLE, UpdateTableRequest.class, LoadTableResponse.class), + DROP_TABLE(HTTPMethod.DELETE, ResourcePaths.V1_TABLE), + RENAME_TABLE(HTTPMethod.POST, ResourcePaths.V1_TABLE_RENAME, RenameTableRequest.class, null), REPORT_METRICS( + HTTPMethod.POST, ResourcePaths.V1_TABLE_METRICS, ReportMetricsRequest.class, null), + COMMIT_TRANSACTION( HTTPMethod.POST, - "v1/namespaces/{namespace}/tables/{name}/metrics", - ReportMetricsRequest.class, + ResourcePaths.V1_TRANSACTIONS_COMMIT, + CommitTransactionRequest.class, null), - COMMIT_TRANSACTION( - HTTPMethod.POST, "v1/transactions/commit", CommitTransactionRequest.class, null), - LIST_VIEWS(HTTPMethod.GET, "v1/namespaces/{namespace}/views", null, ListTablesResponse.class), - LOAD_VIEW( - HTTPMethod.GET, "v1/namespaces/{namespace}/views/{name}", null, LoadViewResponse.class), + LIST_VIEWS(HTTPMethod.GET, ResourcePaths.V1_VIEWS, null, ListTablesResponse.class), + LOAD_VIEW(HTTPMethod.GET, ResourcePaths.V1_VIEW, null, LoadViewResponse.class), CREATE_VIEW( - HTTPMethod.POST, - "v1/namespaces/{namespace}/views", - CreateViewRequest.class, - LoadViewResponse.class), + HTTPMethod.POST, ResourcePaths.V1_VIEWS, CreateViewRequest.class, LoadViewResponse.class), UPDATE_VIEW( - HTTPMethod.POST, - "v1/namespaces/{namespace}/views/{name}", - UpdateTableRequest.class, - LoadViewResponse.class), - RENAME_VIEW(HTTPMethod.POST, "v1/views/rename", RenameTableRequest.class, null), - DROP_VIEW(HTTPMethod.DELETE, "v1/namespaces/{namespace}/views/{name}"); + HTTPMethod.POST, ResourcePaths.V1_VIEW, UpdateTableRequest.class, LoadViewResponse.class), + RENAME_VIEW(HTTPMethod.POST, ResourcePaths.V1_VIEW_RENAME, RenameTableRequest.class, null), + DROP_VIEW(HTTPMethod.DELETE, ResourcePaths.V1_VIEW); private final HTTPMethod method; private final int requiredLength; @@ -177,6 +169,7 @@ enum Route { private final Map variables; private final Class requestClass; private final Class responseClass; + private final String resourcePath; Route(HTTPMethod method, String pattern) { this(method, pattern, null, null); @@ -188,9 +181,11 @@ enum Route { Class requestClass, Class responseClass) { this.method = method; + this.resourcePath = pattern; // parse the pattern into requirements and variables - List parts = SLASH.splitToList(pattern); + List parts = + SLASH.splitToList(pattern.replaceFirst("/v1/", "v1/").replace("/{prefix}", "")); ImmutableMap.Builder requirementsBuilder = ImmutableMap.builder(); ImmutableMap.Builder variablesBuilder = ImmutableMap.builder(); for (int pos = 0; pos < parts.size(); pos += 1) { @@ -245,6 +240,14 @@ public Class requestClass() { public Class responseClass() { return responseClass; } + + HTTPMethod method() { + return method; + } + + String resourcePath() { + return resourcePath; + } } private static OAuthTokenResponse handleOAuthRequest(Object body) { @@ -282,7 +285,14 @@ public T handleRequest( return castResponse(responseType, handleOAuthRequest(body)); case CONFIG: - return castResponse(responseType, ConfigResponse.builder().build()); + return castResponse( + responseType, + ConfigResponse.builder() + .withEndpoints( + Arrays.stream(Route.values()) + .map(r -> Endpoint.create(r.method.name(), r.resourcePath)) + .collect(Collectors.toList())) + .build()); case LIST_NAMESPACES: if (asNamespaceCatalog != null) { @@ -371,16 +381,16 @@ public T handleRequest( case DROP_TABLE: { if (PropertyUtil.propertyAsBoolean(vars, "purgeRequested", false)) { - CatalogHandlers.purgeTable(catalog, identFromPathVars(vars)); + CatalogHandlers.purgeTable(catalog, tableIdentFromPathVars(vars)); } else { - CatalogHandlers.dropTable(catalog, identFromPathVars(vars)); + CatalogHandlers.dropTable(catalog, tableIdentFromPathVars(vars)); } return null; } case LOAD_TABLE: { - TableIdentifier ident = identFromPathVars(vars); + TableIdentifier ident = tableIdentFromPathVars(vars); return castResponse(responseType, CatalogHandlers.loadTable(catalog, ident)); } @@ -394,7 +404,7 @@ public T handleRequest( case UPDATE_TABLE: { - TableIdentifier ident = identFromPathVars(vars); + TableIdentifier ident = tableIdentFromPathVars(vars); UpdateTableRequest request = castRequest(UpdateTableRequest.class, body); return castResponse(responseType, CatalogHandlers.updateTable(catalog, ident, request)); } @@ -452,7 +462,7 @@ public T handleRequest( case LOAD_VIEW: { if (null != asViewCatalog) { - TableIdentifier ident = identFromPathVars(vars); + TableIdentifier ident = viewIdentFromPathVars(vars); return castResponse(responseType, CatalogHandlers.loadView(asViewCatalog, ident)); } break; @@ -461,7 +471,7 @@ public T handleRequest( case UPDATE_VIEW: { if (null != asViewCatalog) { - TableIdentifier ident = identFromPathVars(vars); + TableIdentifier ident = viewIdentFromPathVars(vars); UpdateTableRequest request = castRequest(UpdateTableRequest.class, body); return castResponse( responseType, CatalogHandlers.updateView(asViewCatalog, ident, request)); @@ -482,7 +492,7 @@ public T handleRequest( case DROP_VIEW: { if (null != asViewCatalog) { - CatalogHandlers.dropView(asViewCatalog, identFromPathVars(vars)); + CatalogHandlers.dropView(asViewCatalog, viewIdentFromPathVars(vars)); return null; } break; @@ -668,8 +678,13 @@ private static Namespace namespaceFromPathVars(Map pathVars) { return RESTUtil.decodeNamespace(pathVars.get("namespace")); } - private static TableIdentifier identFromPathVars(Map pathVars) { + private static TableIdentifier tableIdentFromPathVars(Map pathVars) { + return TableIdentifier.of( + namespaceFromPathVars(pathVars), RESTUtil.decodeString(pathVars.get("table"))); + } + + private static TableIdentifier viewIdentFromPathVars(Map pathVars) { return TableIdentifier.of( - namespaceFromPathVars(pathVars), RESTUtil.decodeString(pathVars.get("name"))); + namespaceFromPathVars(pathVars), RESTUtil.decodeString(pathVars.get("view"))); } } diff --git a/core/src/test/java/org/apache/iceberg/rest/TestEndpoint.java b/core/src/test/java/org/apache/iceberg/rest/TestEndpoint.java new file mode 100644 index 000000000000..1873d8799894 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/rest/TestEndpoint.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatCode; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +public class TestEndpoint { + @Test + public void invalidValues() { + assertThatThrownBy(() -> Endpoint.create(null, "endpoint")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid HTTP method: null or empty"); + + assertThatThrownBy(() -> Endpoint.create("", "endpoint")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid HTTP method: null or empty"); + + assertThatThrownBy(() -> Endpoint.create("GET", null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid path: null or empty"); + + assertThatThrownBy(() -> Endpoint.create("GET", "")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid path: null or empty"); + + assertThatThrownBy(() -> Endpoint.create("invalid", "/")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("No enum constant org.apache.hc.core5.http.Method.INVALID"); + } + + @ParameterizedTest + @ValueSource(strings = {"/path", " GET /path", "GET /path ", "GET /path", "GET /path /other"}) + public void invalidFromString(String endpoint) { + assertThatThrownBy(() -> Endpoint.fromString(endpoint)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Invalid endpoint (must consist of two elements separated by a single space): %s", + endpoint); + } + + @Test + public void validFromString() { + Endpoint endpoint = Endpoint.fromString("GET /path"); + assertThat(endpoint.httpMethod()).isEqualTo("GET"); + assertThat(endpoint.path()).isEqualTo("/path"); + } + + @Test + public void toStringRepresentation() { + assertThat(Endpoint.create("POST", "/path/of/resource")) + .asString() + .isEqualTo("POST /path/of/resource"); + assertThat(Endpoint.create("GET", "/")).asString().isEqualTo("GET /"); + assertThat(Endpoint.create("PuT", "/")).asString().isEqualTo("PUT /"); + assertThat(Endpoint.create("PUT", "/namespaces/{namespace}/{x}")) + .asString() + .isEqualTo("PUT /namespaces/{namespace}/{x}"); + } + + @Test + public void supportedEndpoints() { + assertThatCode( + () -> Endpoint.check(ImmutableSet.of(Endpoint.V1_LOAD_TABLE), Endpoint.V1_LOAD_TABLE)) + .doesNotThrowAnyException(); + + assertThatCode( + () -> + Endpoint.check( + ImmutableSet.of(Endpoint.V1_LOAD_TABLE, Endpoint.V1_LOAD_VIEW), + Endpoint.V1_LOAD_TABLE)) + .doesNotThrowAnyException(); + } + + @Test + public void unsupportedEndpoints() { + assertThatThrownBy(() -> Endpoint.check(ImmutableSet.of(), Endpoint.V1_LOAD_TABLE)) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage("Server does not support endpoint: %s", Endpoint.V1_LOAD_TABLE); + + assertThatThrownBy( + () -> Endpoint.check(ImmutableSet.of(Endpoint.V1_LOAD_VIEW), Endpoint.V1_LOAD_TABLE)) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage("Server does not support endpoint: %s", Endpoint.V1_LOAD_TABLE); + } +} diff --git a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java index 1c15cfab43a3..06008761eac1 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java @@ -2341,13 +2341,13 @@ public void testInvalidPageSize() { RESTSessionCatalog.REST_PAGE_SIZE)); } - @Test - public void testPaginationForListNamespaces() { + @ParameterizedTest + @ValueSource(ints = {21, 30}) + public void testPaginationForListNamespaces(int numberOfItems) { RESTCatalogAdapter adapter = Mockito.spy(new RESTCatalogAdapter(backendCatalog)); RESTCatalog catalog = new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) -> adapter); catalog.initialize("test", ImmutableMap.of(RESTSessionCatalog.REST_PAGE_SIZE, "10")); - int numberOfItems = 30; String namespaceName = "newdb"; // create several namespaces for listing and verify @@ -2403,13 +2403,13 @@ public void testPaginationForListNamespaces() { eq(ListNamespacesResponse.class)); } - @Test - public void testPaginationForListTables() { + @ParameterizedTest + @ValueSource(ints = {21, 30}) + public void testPaginationForListTables(int numberOfItems) { RESTCatalogAdapter adapter = Mockito.spy(new RESTCatalogAdapter(backendCatalog)); RESTCatalog catalog = new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) -> adapter); catalog.initialize("test", ImmutableMap.of(RESTSessionCatalog.REST_PAGE_SIZE, "10")); - int numberOfItems = 30; String namespaceName = "newdb"; String tableName = "newtable"; catalog.createNamespace(Namespace.of(namespaceName)); diff --git a/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java index db0969620dc9..85ccdc8f5ddd 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java @@ -50,18 +50,19 @@ import org.eclipse.jetty.servlet.ServletHolder; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import org.mockito.Mockito; public class TestRESTViewCatalog extends ViewCatalogTests { private static final ObjectMapper MAPPER = RESTObjectMapper.mapper(); - @TempDir private Path temp; + @TempDir protected Path temp; - private RESTCatalog restCatalog; - private InMemoryCatalog backendCatalog; - private Server httpServer; + protected RESTCatalog restCatalog; + protected InMemoryCatalog backendCatalog; + protected Server httpServer; @BeforeEach public void createCatalog() throws Exception { @@ -153,14 +154,14 @@ public void closeCatalog() throws Exception { } } - @Test - public void testPaginationForListViews() { + @ParameterizedTest + @ValueSource(ints = {21, 30}) + public void testPaginationForListViews(int numberOfItems) { RESTCatalogAdapter adapter = Mockito.spy(new RESTCatalogAdapter(backendCatalog)); RESTCatalog catalog = new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) -> adapter); catalog.initialize("test", ImmutableMap.of(RESTSessionCatalog.REST_PAGE_SIZE, "10")); - int numberOfItems = 30; String namespaceName = "newdb"; String viewName = "newview"; diff --git a/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalogWithAssumedViewSupport.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalogWithAssumedViewSupport.java new file mode 100644 index 000000000000..3d7d64ddb794 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalogWithAssumedViewSupport.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest; + +import static org.apache.iceberg.rest.RESTCatalogAdapter.Route.CONFIG; + +import java.io.File; +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.catalog.SessionCatalog; +import org.apache.iceberg.inmemory.InMemoryCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.rest.responses.ConfigResponse; +import org.eclipse.jetty.server.Server; +import org.eclipse.jetty.server.handler.gzip.GzipHandler; +import org.eclipse.jetty.servlet.ServletContextHandler; +import org.eclipse.jetty.servlet.ServletHolder; +import org.junit.jupiter.api.BeforeEach; + +public class TestRESTViewCatalogWithAssumedViewSupport extends TestRESTViewCatalog { + + @BeforeEach + public void createCatalog() throws Exception { + File warehouse = temp.toFile(); + + this.backendCatalog = new InMemoryCatalog(); + this.backendCatalog.initialize( + "in-memory", + ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse.getAbsolutePath())); + + RESTCatalogAdapter adaptor = + new RESTCatalogAdapter(backendCatalog) { + + @Override + public T handleRequest( + Route route, Map vars, Object body, Class responseType) { + if (CONFIG == route) { + // simulate a legacy server that doesn't send back supported endpoints + return castResponse(responseType, ConfigResponse.builder().build()); + } + + return super.handleRequest(route, vars, body, responseType); + } + }; + + ServletContextHandler servletContext = + new ServletContextHandler(ServletContextHandler.NO_SESSIONS); + servletContext.setContextPath("/"); + servletContext.addServlet(new ServletHolder(new RESTCatalogServlet(adaptor)), "/*"); + servletContext.setHandler(new GzipHandler()); + + this.httpServer = new Server(0); + httpServer.setHandler(servletContext); + httpServer.start(); + + SessionCatalog.SessionContext context = + new SessionCatalog.SessionContext( + UUID.randomUUID().toString(), + "user", + ImmutableMap.of("credential", "user:12345"), + ImmutableMap.of()); + + this.restCatalog = + new RESTCatalog( + context, + (config) -> HTTPClient.builder(config).uri(config.get(CatalogProperties.URI)).build()); + restCatalog.initialize( + "prod", + ImmutableMap.of( + CatalogProperties.URI, + httpServer.getURI().toString(), + "credential", + "catalog:12345", + // assume that the server supports view endpoints + RESTSessionCatalog.VIEW_ENDPOINTS_SUPPORTED, + "true")); + } +} diff --git a/core/src/test/java/org/apache/iceberg/rest/credentials/TestCredentialParser.java b/core/src/test/java/org/apache/iceberg/rest/credentials/TestCredentialParser.java new file mode 100644 index 000000000000..a48fd7353b98 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/rest/credentials/TestCredentialParser.java @@ -0,0 +1,143 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest.credentials; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import com.fasterxml.jackson.databind.JsonNode; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.Test; + +public class TestCredentialParser { + @Test + public void nullAndEmptyCheck() { + assertThatThrownBy(() -> CredentialParser.toJson(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid credential: null"); + + assertThatThrownBy(() -> CredentialParser.fromJson((JsonNode) null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse credential from null object"); + } + + @Test + public void invalidOrMissingFields() { + assertThatThrownBy(() -> CredentialParser.fromJson("{}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing string: prefix"); + + assertThatThrownBy(() -> CredentialParser.fromJson("{\"prefix\": \"y\"}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing map: config"); + + assertThatThrownBy( + () -> CredentialParser.fromJson("{\"prefix\": \"\", \"config\": {\"x\": \"23\"}}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid prefix: must be non-empty"); + + assertThatThrownBy(() -> CredentialParser.fromJson("{\"prefix\": \"s3\", \"config\": {}}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid config: must be non-empty"); + } + + @Test + public void s3Credential() { + Credential credential = + ImmutableCredential.builder() + .prefix("s3://custom-uri") + .config( + ImmutableMap.of( + "s3.access-key-id", + "keyId", + "s3.secret-access-key", + "accessKey", + "s3.session-token", + "sessionToken")) + .build(); + + String expectedJson = + "{\n" + + " \"prefix\" : \"s3://custom-uri\",\n" + + " \"config\" : {\n" + + " \"s3.access-key-id\" : \"keyId\",\n" + + " \"s3.secret-access-key\" : \"accessKey\",\n" + + " \"s3.session-token\" : \"sessionToken\"\n" + + " }\n" + + "}"; + + String json = CredentialParser.toJson(credential, true); + assertThat(json).isEqualTo(expectedJson); + assertThat(CredentialParser.toJson(CredentialParser.fromJson(json), true)) + .isEqualTo(expectedJson); + } + + @Test + public void gcsCredential() { + Credential credential = + ImmutableCredential.builder() + .prefix("gs://custom-uri") + .config( + ImmutableMap.of( + "gcs.oauth2.token", "gcsToken", "gcs.oauth2.token-expires-at", "1000")) + .build(); + + String expectedJson = + "{\n" + + " \"prefix\" : \"gs://custom-uri\",\n" + + " \"config\" : {\n" + + " \"gcs.oauth2.token\" : \"gcsToken\",\n" + + " \"gcs.oauth2.token-expires-at\" : \"1000\"\n" + + " }\n" + + "}"; + + String json = CredentialParser.toJson(credential, true); + assertThat(json).isEqualTo(expectedJson); + assertThat(CredentialParser.toJson(CredentialParser.fromJson(json), true)) + .isEqualTo(expectedJson); + } + + @Test + public void adlsCredential() { + Credential credential = + ImmutableCredential.builder() + .prefix("abfs://custom-uri") + .config( + ImmutableMap.of( + "adls.sas-token.account", + "sasToken", + "adls.auth.shared-key.account.key", + "accountKey")) + .build(); + + String expectedJson = + "{\n" + + " \"prefix\" : \"abfs://custom-uri\",\n" + + " \"config\" : {\n" + + " \"adls.sas-token.account\" : \"sasToken\",\n" + + " \"adls.auth.shared-key.account.key\" : \"accountKey\"\n" + + " }\n" + + "}"; + + String json = CredentialParser.toJson(credential, true); + assertThat(json).isEqualTo(expectedJson); + assertThat(CredentialParser.toJson(CredentialParser.fromJson(json), true)) + .isEqualTo(expectedJson); + } +} diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestConfigResponseParser.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestConfigResponseParser.java index ec4c793c279f..81ec7cc5585c 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestConfigResponseParser.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestConfigResponseParser.java @@ -22,8 +22,11 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import com.fasterxml.jackson.databind.JsonNode; +import java.util.List; import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.rest.Endpoint; import org.junit.jupiter.api.Test; public class TestConfigResponseParser { @@ -106,6 +109,27 @@ public void overridesOnly() { .isEqualTo(expectedJson); } + @Test + public void endpointsOnly() { + List endpoints = + ImmutableList.of( + Endpoint.V1_LOAD_NAMESPACE, Endpoint.V1_LIST_NAMESPACES, Endpoint.V1_CREATE_NAMESPACE); + ConfigResponse response = ConfigResponse.builder().withEndpoints(endpoints).build(); + + String expectedJson = + "{\n" + + " \"defaults\" : { },\n" + + " \"overrides\" : { },\n" + + " \"endpoints\" : [ \"GET /v1/{prefix}/namespaces/{namespace}\", \"GET /v1/{prefix}/namespaces\", \"POST /v1/{prefix}/namespaces\" ]\n" + + "}"; + + String json = ConfigResponseParser.toJson(response, true); + assertThat(json).isEqualTo(expectedJson); + assertThat(ConfigResponseParser.toJson(ConfigResponseParser.fromJson(json), true)) + .isEqualTo(expectedJson); + assertThat(ConfigResponseParser.fromJson(json).endpoints()).isEqualTo(response.endpoints()); + } + @Test public void roundTripSerde() { Map defaults = Maps.newHashMap(); @@ -135,4 +159,59 @@ public void roundTripSerde() { assertThat(ConfigResponseParser.toJson(ConfigResponseParser.fromJson(json), true)) .isEqualTo(expectedJson); } + + @Test + public void invalidEndpoint() { + assertThatThrownBy( + () -> + ConfigResponseParser.fromJson( + "{\"endpoints\":[\"GET_v1/namespaces/{namespace}\"]}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Invalid endpoint (must consist of two elements separated by a single space): GET_v1/namespaces/{namespace}"); + + assertThatThrownBy( + () -> + ConfigResponseParser.fromJson( + "{\"endpoints\":[\"GET v1/namespaces/{namespace} INVALID\"]}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Invalid endpoint (must consist of two elements separated by a single space): GET v1/namespaces/{namespace} INVALID"); + } + + @Test + public void roundTripSerdeWithEndpoints() { + Map defaults = Maps.newHashMap(); + defaults.put("key1", "1"); + defaults.put("key2", null); + + Map overrides = Maps.newHashMap(); + overrides.put("key3", "23"); + overrides.put("key4", null); + + ConfigResponse response = + ConfigResponse.builder() + .withDefaults(defaults) + .withOverrides(overrides) + .withEndpoints(ImmutableList.of(Endpoint.V1_LOAD_TABLE, Endpoint.V1_LOAD_VIEW)) + .build(); + + String expectedJson = + "{\n" + + " \"defaults\" : {\n" + + " \"key1\" : \"1\",\n" + + " \"key2\" : null\n" + + " },\n" + + " \"overrides\" : {\n" + + " \"key3\" : \"23\",\n" + + " \"key4\" : null\n" + + " },\n" + + " \"endpoints\" : [ \"GET /v1/{prefix}/namespaces/{namespace}/tables/{table}\", \"GET /v1/{prefix}/namespaces/{namespace}/views/{view}\" ]\n" + + "}"; + + String json = ConfigResponseParser.toJson(response, true); + assertThat(json).isEqualTo(expectedJson); + assertThat(ConfigResponseParser.toJson(ConfigResponseParser.fromJson(json), true)) + .isEqualTo(expectedJson); + } } diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadCredentialsResponseParser.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadCredentialsResponseParser.java new file mode 100644 index 000000000000..f2e723da2540 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadCredentialsResponseParser.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest.responses; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import com.fasterxml.jackson.databind.JsonNode; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.rest.credentials.ImmutableCredential; +import org.junit.jupiter.api.Test; + +public class TestLoadCredentialsResponseParser { + @Test + public void nullCheck() { + assertThatThrownBy(() -> LoadCredentialsResponseParser.toJson(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid load credentials response: null"); + + assertThatThrownBy(() -> LoadCredentialsResponseParser.fromJson((JsonNode) null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse load credentials response from null object"); + } + + @Test + public void missingFields() { + assertThatThrownBy(() -> LoadCredentialsResponseParser.fromJson("{}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing field: storage-credentials"); + + assertThatThrownBy(() -> LoadCredentialsResponseParser.fromJson("{\"x\": \"val\"}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing field: storage-credentials"); + } + + @Test + public void roundTripSerde() { + LoadCredentialsResponse response = + ImmutableLoadCredentialsResponse.builder() + .addCredentials( + ImmutableCredential.builder() + .prefix("s3://custom-uri") + .config( + ImmutableMap.of( + "s3.access-key-id", + "keyId", + "s3.secret-access-key", + "accessKey", + "s3.session-token", + "sessionToken")) + .build()) + .addCredentials( + ImmutableCredential.builder() + .prefix("gs://custom-uri") + .config( + ImmutableMap.of( + "gcs.oauth2.token", "gcsToken1", "gcs.oauth2.token-expires-at", "1000")) + .build()) + .addCredentials( + ImmutableCredential.builder() + .prefix("gs") + .config( + ImmutableMap.of( + "gcs.oauth2.token", "gcsToken2", "gcs.oauth2.token-expires-at", "2000")) + .build()) + .build(); + + String expectedJson = + "{\n" + + " \"storage-credentials\" : [ {\n" + + " \"prefix\" : \"s3://custom-uri\",\n" + + " \"config\" : {\n" + + " \"s3.access-key-id\" : \"keyId\",\n" + + " \"s3.secret-access-key\" : \"accessKey\",\n" + + " \"s3.session-token\" : \"sessionToken\"\n" + + " }\n" + + " }, {\n" + + " \"prefix\" : \"gs://custom-uri\",\n" + + " \"config\" : {\n" + + " \"gcs.oauth2.token\" : \"gcsToken1\",\n" + + " \"gcs.oauth2.token-expires-at\" : \"1000\"\n" + + " }\n" + + " }, {\n" + + " \"prefix\" : \"gs\",\n" + + " \"config\" : {\n" + + " \"gcs.oauth2.token\" : \"gcsToken2\",\n" + + " \"gcs.oauth2.token-expires-at\" : \"2000\"\n" + + " }\n" + + " } ]\n" + + "}"; + + String json = LoadCredentialsResponseParser.toJson(response, true); + assertThat(json).isEqualTo(expectedJson); + assertThat(LoadCredentialsResponseParser.fromJson(json)).isEqualTo(response); + } +} diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponseParser.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponseParser.java new file mode 100644 index 000000000000..cc6f4cfc74d7 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponseParser.java @@ -0,0 +1,327 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest.responses; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import com.fasterxml.jackson.databind.JsonNode; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.rest.credentials.ImmutableCredential; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestLoadTableResponseParser { + + @Test + public void nullAndEmptyCheck() { + assertThatThrownBy(() -> LoadTableResponseParser.toJson(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid load table response: null"); + + assertThatThrownBy(() -> LoadTableResponseParser.fromJson((JsonNode) null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse load table response from null object"); + + assertThatThrownBy(() -> LoadTableResponseParser.fromJson("{}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing field: metadata"); + } + + @Test + public void missingFields() { + assertThatThrownBy( + () -> LoadTableResponseParser.fromJson("{\"metadata-location\": \"custom-location\"}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing field: metadata"); + } + + @Test + public void roundTripSerde() { + String uuid = "386b9f01-002b-4d8c-b77f-42c3fd3b7c9b"; + TableMetadata metadata = + TableMetadata.buildFromEmpty() + .assignUUID(uuid) + .setLocation("location") + .setCurrentSchema( + new Schema(Types.NestedField.required(1, "x", Types.LongType.get())), 1) + .addPartitionSpec(PartitionSpec.unpartitioned()) + .addSortOrder(SortOrder.unsorted()) + .discardChanges() + .withMetadataLocation("metadata-location") + .build(); + + LoadTableResponse response = LoadTableResponse.builder().withTableMetadata(metadata).build(); + + String expectedJson = + String.format( + "{\n" + + " \"metadata-location\" : \"metadata-location\",\n" + + " \"metadata\" : {\n" + + " \"format-version\" : 2,\n" + + " \"table-uuid\" : \"386b9f01-002b-4d8c-b77f-42c3fd3b7c9b\",\n" + + " \"location\" : \"location\",\n" + + " \"last-sequence-number\" : 0,\n" + + " \"last-updated-ms\" : %d,\n" + + " \"last-column-id\" : 1,\n" + + " \"current-schema-id\" : 0,\n" + + " \"schemas\" : [ {\n" + + " \"type\" : \"struct\",\n" + + " \"schema-id\" : 0,\n" + + " \"fields\" : [ {\n" + + " \"id\" : 1,\n" + + " \"name\" : \"x\",\n" + + " \"required\" : true,\n" + + " \"type\" : \"long\"\n" + + " } ]\n" + + " } ],\n" + + " \"default-spec-id\" : 0,\n" + + " \"partition-specs\" : [ {\n" + + " \"spec-id\" : 0,\n" + + " \"fields\" : [ ]\n" + + " } ],\n" + + " \"last-partition-id\" : 999,\n" + + " \"default-sort-order-id\" : 0,\n" + + " \"sort-orders\" : [ {\n" + + " \"order-id\" : 0,\n" + + " \"fields\" : [ ]\n" + + " } ],\n" + + " \"properties\" : { },\n" + + " \"current-snapshot-id\" : -1,\n" + + " \"refs\" : { },\n" + + " \"snapshots\" : [ ],\n" + + " \"statistics\" : [ ],\n" + + " \"partition-statistics\" : [ ],\n" + + " \"snapshot-log\" : [ ],\n" + + " \"metadata-log\" : [ ]\n" + + " }\n" + + "}", + metadata.lastUpdatedMillis()); + + String json = LoadTableResponseParser.toJson(response, true); + assertThat(json).isEqualTo(expectedJson); + // can't do an equality comparison because Schema doesn't implement equals/hashCode + assertThat(LoadTableResponseParser.toJson(LoadTableResponseParser.fromJson(json), true)) + .isEqualTo(expectedJson); + } + + @Test + public void roundTripSerdeWithConfig() { + String uuid = "386b9f01-002b-4d8c-b77f-42c3fd3b7c9b"; + TableMetadata metadata = + TableMetadata.buildFromEmpty() + .assignUUID(uuid) + .setLocation("location") + .setCurrentSchema( + new Schema(Types.NestedField.required(1, "x", Types.LongType.get())), 1) + .addPartitionSpec(PartitionSpec.unpartitioned()) + .addSortOrder(SortOrder.unsorted()) + .discardChanges() + .withMetadataLocation("metadata-location") + .build(); + + LoadTableResponse response = + LoadTableResponse.builder() + .withTableMetadata(metadata) + .addAllConfig(ImmutableMap.of("key1", "val1", "key2", "val2")) + .build(); + + String expectedJson = + String.format( + "{\n" + + " \"metadata-location\" : \"metadata-location\",\n" + + " \"metadata\" : {\n" + + " \"format-version\" : 2,\n" + + " \"table-uuid\" : \"386b9f01-002b-4d8c-b77f-42c3fd3b7c9b\",\n" + + " \"location\" : \"location\",\n" + + " \"last-sequence-number\" : 0,\n" + + " \"last-updated-ms\" : %d,\n" + + " \"last-column-id\" : 1,\n" + + " \"current-schema-id\" : 0,\n" + + " \"schemas\" : [ {\n" + + " \"type\" : \"struct\",\n" + + " \"schema-id\" : 0,\n" + + " \"fields\" : [ {\n" + + " \"id\" : 1,\n" + + " \"name\" : \"x\",\n" + + " \"required\" : true,\n" + + " \"type\" : \"long\"\n" + + " } ]\n" + + " } ],\n" + + " \"default-spec-id\" : 0,\n" + + " \"partition-specs\" : [ {\n" + + " \"spec-id\" : 0,\n" + + " \"fields\" : [ ]\n" + + " } ],\n" + + " \"last-partition-id\" : 999,\n" + + " \"default-sort-order-id\" : 0,\n" + + " \"sort-orders\" : [ {\n" + + " \"order-id\" : 0,\n" + + " \"fields\" : [ ]\n" + + " } ],\n" + + " \"properties\" : { },\n" + + " \"current-snapshot-id\" : -1,\n" + + " \"refs\" : { },\n" + + " \"snapshots\" : [ ],\n" + + " \"statistics\" : [ ],\n" + + " \"partition-statistics\" : [ ],\n" + + " \"snapshot-log\" : [ ],\n" + + " \"metadata-log\" : [ ]\n" + + " },\n" + + " \"config\" : {\n" + + " \"key1\" : \"val1\",\n" + + " \"key2\" : \"val2\"\n" + + " }\n" + + "}", + metadata.lastUpdatedMillis()); + + String json = LoadTableResponseParser.toJson(response, true); + assertThat(json).isEqualTo(expectedJson); + // can't do an equality comparison because Schema doesn't implement equals/hashCode + assertThat(LoadTableResponseParser.toJson(LoadTableResponseParser.fromJson(json), true)) + .isEqualTo(expectedJson); + } + + @Test + public void roundTripSerdeWithCredentials() { + String uuid = "386b9f01-002b-4d8c-b77f-42c3fd3b7c9b"; + TableMetadata metadata = + TableMetadata.buildFromEmpty() + .assignUUID(uuid) + .setLocation("location") + .setCurrentSchema( + new Schema(Types.NestedField.required(1, "x", Types.LongType.get())), 1) + .addPartitionSpec(PartitionSpec.unpartitioned()) + .addSortOrder(SortOrder.unsorted()) + .discardChanges() + .withMetadataLocation("metadata-location") + .build(); + + LoadTableResponse response = + LoadTableResponse.builder() + .withTableMetadata(metadata) + .addAllConfig(ImmutableMap.of("key1", "val1", "key2", "val2")) + .addCredential( + ImmutableCredential.builder() + .prefix("s3://custom-uri") + .config( + ImmutableMap.of( + "s3.access-key-id", + "keyId", + "s3.secret-access-key", + "accessKey", + "s3.session-token", + "sessionToken")) + .build()) + .addCredential( + ImmutableCredential.builder() + .prefix("gs://custom-uri") + .config( + ImmutableMap.of( + "gcs.oauth2.token", "gcsToken1", "gcs.oauth2.token-expires-at", "1000")) + .build()) + .addCredential( + ImmutableCredential.builder() + .prefix("gs") + .config( + ImmutableMap.of( + "gcs.oauth2.token", "gcsToken2", "gcs.oauth2.token-expires-at", "2000")) + .build()) + .build(); + + String expectedJson = + String.format( + "{\n" + + " \"metadata-location\" : \"metadata-location\",\n" + + " \"metadata\" : {\n" + + " \"format-version\" : 2,\n" + + " \"table-uuid\" : \"386b9f01-002b-4d8c-b77f-42c3fd3b7c9b\",\n" + + " \"location\" : \"location\",\n" + + " \"last-sequence-number\" : 0,\n" + + " \"last-updated-ms\" : %s,\n" + + " \"last-column-id\" : 1,\n" + + " \"current-schema-id\" : 0,\n" + + " \"schemas\" : [ {\n" + + " \"type\" : \"struct\",\n" + + " \"schema-id\" : 0,\n" + + " \"fields\" : [ {\n" + + " \"id\" : 1,\n" + + " \"name\" : \"x\",\n" + + " \"required\" : true,\n" + + " \"type\" : \"long\"\n" + + " } ]\n" + + " } ],\n" + + " \"default-spec-id\" : 0,\n" + + " \"partition-specs\" : [ {\n" + + " \"spec-id\" : 0,\n" + + " \"fields\" : [ ]\n" + + " } ],\n" + + " \"last-partition-id\" : 999,\n" + + " \"default-sort-order-id\" : 0,\n" + + " \"sort-orders\" : [ {\n" + + " \"order-id\" : 0,\n" + + " \"fields\" : [ ]\n" + + " } ],\n" + + " \"properties\" : { },\n" + + " \"current-snapshot-id\" : -1,\n" + + " \"refs\" : { },\n" + + " \"snapshots\" : [ ],\n" + + " \"statistics\" : [ ],\n" + + " \"partition-statistics\" : [ ],\n" + + " \"snapshot-log\" : [ ],\n" + + " \"metadata-log\" : [ ]\n" + + " },\n" + + " \"config\" : {\n" + + " \"key1\" : \"val1\",\n" + + " \"key2\" : \"val2\"\n" + + " },\n" + + " \"storage-credentials\" : [ {\n" + + " \"prefix\" : \"s3://custom-uri\",\n" + + " \"config\" : {\n" + + " \"s3.access-key-id\" : \"keyId\",\n" + + " \"s3.secret-access-key\" : \"accessKey\",\n" + + " \"s3.session-token\" : \"sessionToken\"\n" + + " }\n" + + " }, {\n" + + " \"prefix\" : \"gs://custom-uri\",\n" + + " \"config\" : {\n" + + " \"gcs.oauth2.token\" : \"gcsToken1\",\n" + + " \"gcs.oauth2.token-expires-at\" : \"1000\"\n" + + " }\n" + + " }, {\n" + + " \"prefix\" : \"gs\",\n" + + " \"config\" : {\n" + + " \"gcs.oauth2.token\" : \"gcsToken2\",\n" + + " \"gcs.oauth2.token-expires-at\" : \"2000\"\n" + + " }\n" + + " } ]\n" + + "}", + metadata.lastUpdatedMillis()); + + String json = LoadTableResponseParser.toJson(response, true); + assertThat(json).isEqualTo(expectedJson); + // can't do an equality comparison because Schema doesn't implement equals/hashCode + assertThat(LoadTableResponseParser.toJson(LoadTableResponseParser.fromJson(json), true)) + .isEqualTo(expectedJson); + } +} diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadViewResponseParser.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadViewResponseParser.java index f3de08cd2912..086db0fec8b4 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadViewResponseParser.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadViewResponseParser.java @@ -25,6 +25,7 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.rest.credentials.ImmutableCredential; import org.apache.iceberg.types.Types; import org.apache.iceberg.view.ImmutableViewVersion; import org.apache.iceberg.view.ViewMetadata; @@ -245,4 +246,114 @@ public void roundTripSerdeWithConfig() { assertThat(LoadViewResponseParser.toJson(LoadViewResponseParser.fromJson(json), true)) .isEqualTo(expectedJson); } + + @Test + public void roundTripSerdeWithCredentials() { + String uuid = "386b9f01-002b-4d8c-b77f-42c3fd3b7c9b"; + ViewMetadata viewMetadata = + ViewMetadata.builder() + .assignUUID(uuid) + .setLocation("location") + .addSchema(new Schema(Types.NestedField.required(1, "x", Types.LongType.get()))) + .addVersion( + ImmutableViewVersion.builder() + .schemaId(0) + .versionId(1) + .timestampMillis(23L) + .defaultNamespace(Namespace.of("ns1")) + .build()) + .setCurrentVersionId(1) + .build(); + + LoadViewResponse response = + ImmutableLoadViewResponse.builder() + .metadata(viewMetadata) + .metadataLocation("custom-location") + .addCredentials( + ImmutableCredential.builder() + .prefix("s3://custom-uri") + .config( + ImmutableMap.of( + "s3.access-key-id", + "keyId", + "s3.secret-access-key", + "accessKey", + "s3.session-token", + "sessionToken")) + .build()) + .addCredentials( + ImmutableCredential.builder() + .prefix("gs://custom-uri") + .config( + ImmutableMap.of( + "gcs.oauth2.token", "gcsToken1", "gcs.oauth2.token-expires-at", "1000")) + .build()) + .addCredentials( + ImmutableCredential.builder() + .prefix("gs") + .config( + ImmutableMap.of( + "gcs.oauth2.token", "gcsToken2", "gcs.oauth2.token-expires-at", "2000")) + .build()) + .build(); + + String expectedJson = + "{\n" + + " \"metadata-location\" : \"custom-location\",\n" + + " \"metadata\" : {\n" + + " \"view-uuid\" : \"386b9f01-002b-4d8c-b77f-42c3fd3b7c9b\",\n" + + " \"format-version\" : 1,\n" + + " \"location\" : \"location\",\n" + + " \"schemas\" : [ {\n" + + " \"type\" : \"struct\",\n" + + " \"schema-id\" : 0,\n" + + " \"fields\" : [ {\n" + + " \"id\" : 1,\n" + + " \"name\" : \"x\",\n" + + " \"required\" : true,\n" + + " \"type\" : \"long\"\n" + + " } ]\n" + + " } ],\n" + + " \"current-version-id\" : 1,\n" + + " \"versions\" : [ {\n" + + " \"version-id\" : 1,\n" + + " \"timestamp-ms\" : 23,\n" + + " \"schema-id\" : 0,\n" + + " \"summary\" : { },\n" + + " \"default-namespace\" : [ \"ns1\" ],\n" + + " \"representations\" : [ ]\n" + + " } ],\n" + + " \"version-log\" : [ {\n" + + " \"timestamp-ms\" : 23,\n" + + " \"version-id\" : 1\n" + + " } ]\n" + + " },\n" + + " \"storage-credentials\" : [ {\n" + + " \"prefix\" : \"s3://custom-uri\",\n" + + " \"config\" : {\n" + + " \"s3.access-key-id\" : \"keyId\",\n" + + " \"s3.secret-access-key\" : \"accessKey\",\n" + + " \"s3.session-token\" : \"sessionToken\"\n" + + " }\n" + + " }, {\n" + + " \"prefix\" : \"gs://custom-uri\",\n" + + " \"config\" : {\n" + + " \"gcs.oauth2.token\" : \"gcsToken1\",\n" + + " \"gcs.oauth2.token-expires-at\" : \"1000\"\n" + + " }\n" + + " }, {\n" + + " \"prefix\" : \"gs\",\n" + + " \"config\" : {\n" + + " \"gcs.oauth2.token\" : \"gcsToken2\",\n" + + " \"gcs.oauth2.token-expires-at\" : \"2000\"\n" + + " }\n" + + " } ]\n" + + "}"; + + String json = LoadViewResponseParser.toJson(response, true); + assertThat(json).isEqualTo(expectedJson); + // can't do an equality comparison because Schema doesn't implement equals/hashCode + assertThat(LoadViewResponseParser.toJson(LoadViewResponseParser.fromJson(json), true)) + .isEqualTo(expectedJson); + } } diff --git a/core/src/test/java/org/apache/iceberg/util/TestDataFileSet.java b/core/src/test/java/org/apache/iceberg/util/TestDataFileSet.java new file mode 100644 index 000000000000..0f298ad82e9d --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/util/TestDataFileSet.java @@ -0,0 +1,303 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.util; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Set; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.junit.jupiter.api.Test; + +/** + * Testing {@link DataFileSet} is easier in iceberg-core since the data file builders are located + * here + */ +public class TestDataFileSet { + + private static final DataFile FILE_A = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-a.parquet") + .withFileSizeInBytes(1) + .withRecordCount(1) + .build(); + private static final DataFile FILE_B = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-b.parquet") + .withFileSizeInBytes(2) + .withRecordCount(2) + .build(); + private static final DataFile FILE_C = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-c.parquet") + .withFileSizeInBytes(3) + .withRecordCount(3) + .build(); + private static final DataFile FILE_D = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-d.parquet") + .withFileSizeInBytes(4) + .withRecordCount(4) + .build(); + + @Test + public void emptySet() { + assertThat(DataFileSet.create()).isEmpty(); + assertThat(DataFileSet.create()).doesNotContain(FILE_A, FILE_B, FILE_C); + } + + @Test + public void insertionOrderIsMaintained() { + DataFileSet set = DataFileSet.create(); + set.addAll(ImmutableList.of(FILE_D, FILE_A, FILE_C)); + set.add(FILE_B); + set.add(FILE_D); + + assertThat(set).hasSize(4).containsExactly(FILE_D, FILE_A, FILE_C, FILE_B); + } + + @Test + public void clear() { + DataFileSet set = DataFileSet.of(ImmutableList.of(FILE_A, FILE_B)); + set.clear(); + assertThat(set).isEmpty(); + } + + @Test + public void addAll() { + DataFileSet empty = DataFileSet.create(); + assertThatThrownBy(() -> empty.add(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThatThrownBy(() -> empty.addAll(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid collection: null"); + + assertThatThrownBy(() -> empty.addAll(Collections.singletonList(null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThatThrownBy(() -> empty.addAll(Arrays.asList(FILE_A, null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + DataFileSet set = DataFileSet.create(); + set.addAll(ImmutableList.of(FILE_B, FILE_A, FILE_C, FILE_A)); + assertThat(set).hasSize(3).containsExactly(FILE_B, FILE_A, FILE_C); + } + + @Test + public void contains() { + DataFileSet set = DataFileSet.of(ImmutableList.of(FILE_A, FILE_B)); + assertThatThrownBy(() -> set.contains(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThat(set) + .hasSize(2) + .containsExactly(FILE_A, FILE_B) + .doesNotContain(FILE_C) + .doesNotContain(FILE_D); + + assertThatThrownBy(() -> DataFileSet.of(Arrays.asList(FILE_C, FILE_B, null, FILE_A))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + } + + @Test + public void containsAll() { + DataFileSet set = DataFileSet.of(ImmutableList.of(FILE_A, FILE_B)); + assertThatThrownBy(() -> set.containsAll(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid collection: null"); + + assertThatThrownBy(() -> set.containsAll(Collections.singletonList(null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThatThrownBy(() -> set.containsAll(Arrays.asList(FILE_A, null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThat(set.containsAll(ImmutableList.of(FILE_B, FILE_A))).isTrue(); + assertThat(set.containsAll(ImmutableList.of(FILE_B, FILE_A, FILE_C))).isFalse(); + assertThat(set.containsAll(ImmutableList.of(FILE_B))).isTrue(); + } + + @Test + public void toArray() { + DataFileSet set = DataFileSet.of(ImmutableList.of(FILE_B, FILE_A)); + assertThat(set.toArray()).hasSize(2).containsExactly(FILE_B, FILE_A); + + DataFile[] array = new DataFile[1]; + assertThat(set.toArray(array)).hasSize(2).containsExactly(FILE_B, FILE_A); + + array = new DataFile[0]; + assertThat(set.toArray(array)).hasSize(2).containsExactly(FILE_B, FILE_A); + + array = new DataFile[5]; + assertThat(set.toArray(array)).hasSize(5).containsExactly(FILE_B, FILE_A, null, null, null); + + array = new DataFile[2]; + assertThat(set.toArray(array)).hasSize(2).containsExactly(FILE_B, FILE_A); + } + + @Test + public void retainAll() { + DataFileSet empty = DataFileSet.create(); + assertThatThrownBy(() -> empty.retainAll(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid collection: null"); + + assertThatThrownBy(() -> empty.retainAll(Collections.singletonList(null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThatThrownBy(() -> empty.retainAll(Arrays.asList(FILE_A, null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + DataFileSet set = DataFileSet.of(ImmutableList.of(FILE_A, FILE_B)); + assertThat(set.retainAll(ImmutableList.of(FILE_C, FILE_D, FILE_A))) + .as("Set should have changed") + .isTrue(); + + assertThat(set).hasSize(1).containsExactly(FILE_A); + + set = DataFileSet.of(ImmutableList.of(FILE_A, FILE_B)); + + assertThat(set.retainAll(ImmutableList.of(FILE_B, FILE_A))) + .as("Set should not have changed") + .isFalse(); + + assertThat(set.retainAll(ImmutableList.of(FILE_C, FILE_D))) + .as("Set should have changed") + .isTrue(); + + assertThat(set).isEmpty(); + } + + @Test + public void remove() { + DataFileSet set = DataFileSet.of(ImmutableList.of(FILE_A, FILE_B)); + assertThatThrownBy(() -> set.remove(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + set.remove(FILE_C); + assertThat(set).containsExactly(FILE_A, FILE_B); + set.remove(FILE_B); + assertThat(set).containsExactly(FILE_A); + set.remove(FILE_A); + assertThat(set).isEmpty(); + } + + @Test + public void removeAll() { + DataFileSet empty = DataFileSet.create(); + assertThatThrownBy(() -> empty.removeAll(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid collection: null"); + + assertThatThrownBy(() -> empty.removeAll(Collections.singletonList(null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThatThrownBy(() -> empty.removeAll(Arrays.asList(FILE_A, null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + DataFileSet set = DataFileSet.of(ImmutableList.of(FILE_A, FILE_B)); + + assertThat(set.removeAll(ImmutableList.of(FILE_C, FILE_D, FILE_A))) + .as("Set should have changed") + .isTrue(); + + assertThat(set).hasSize(1).containsExactly(FILE_B); + + set = DataFileSet.of(ImmutableList.of(FILE_A, FILE_B)); + assertThat(set.removeAll(ImmutableList.of(FILE_C, FILE_D))) + .as("Set should not have changed") + .isFalse(); + + assertThat(set.removeAll(ImmutableList.of(FILE_B, FILE_A))) + .as("Set should have changed") + .isTrue(); + + assertThat(set).isEmpty(); + } + + @Test + public void equalsAndHashCode() { + DataFileSet set1 = DataFileSet.create(); + DataFileSet set2 = DataFileSet.create(); + + assertThat(set1).isEqualTo(set2); + assertThat(set1.hashCode()).isEqualTo(set2.hashCode()); + + set1.add(FILE_A); + set1.add(FILE_B); + set1.add(FILE_C); + + // different DataFile instances but all use the same paths as set1 + set2.add( + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath(FILE_A.location()) + .withFileSizeInBytes(10) + .withRecordCount(1) + .build()); + set2.add( + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath(FILE_B.location()) + .withFileSizeInBytes(100) + .withRecordCount(10) + .build()); + set2.add( + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath(FILE_C.location()) + .withFileSizeInBytes(1000) + .withRecordCount(100) + .build()); + + Set set3 = Collections.unmodifiableSet(set2); + + assertThat(set1).isEqualTo(set2).isEqualTo(set3); + assertThat(set1.hashCode()).isEqualTo(set2.hashCode()).isEqualTo(set3.hashCode()); + } + + @Test + public void kryoSerialization() throws Exception { + DataFileSet dataFiles = DataFileSet.of(ImmutableList.of(FILE_C, FILE_B, FILE_A)); + assertThat(TestHelpers.KryoHelpers.roundTripSerialize(dataFiles)).isEqualTo(dataFiles); + } + + @Test + public void javaSerialization() throws Exception { + DataFileSet dataFiles = DataFileSet.of(ImmutableList.of(FILE_C, FILE_B, FILE_A)); + DataFileSet deserialized = TestHelpers.deserialize(TestHelpers.serialize(dataFiles)); + assertThat(deserialized).isEqualTo(dataFiles); + } +} diff --git a/core/src/test/java/org/apache/iceberg/util/TestDeleteFileSet.java b/core/src/test/java/org/apache/iceberg/util/TestDeleteFileSet.java new file mode 100644 index 000000000000..5f4488a3a1d5 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/util/TestDeleteFileSet.java @@ -0,0 +1,321 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.util; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Set; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.junit.jupiter.api.Test; + +/** + * Testing {@link DeleteFileSet} is easier in iceberg-core since the delete file builders are + * located here + */ +public class TestDeleteFileSet { + + private static final DeleteFile FILE_A_DELETES = + FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofPositionDeletes() + .withPath("/path/to/data-a-deletes.parquet") + .withFileSizeInBytes(1) + .withRecordCount(1) + .build(); + private static final DeleteFile FILE_B_DELETES = + FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofPositionDeletes() + .withPath("/path/to/data-b-deletes.parquet") + .withFileSizeInBytes(2) + .withRecordCount(2) + .build(); + private static final DeleteFile FILE_C_DELETES = + FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofPositionDeletes() + .withPath("/path/to/data-c-deletes.parquet") + .withFileSizeInBytes(3) + .withRecordCount(3) + .build(); + private static final DeleteFile FILE_D_DELETES = + FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofPositionDeletes() + .withPath("/path/to/data-d-deletes.parquet") + .withFileSizeInBytes(4) + .withRecordCount(4) + .build(); + + @Test + public void emptySet() { + assertThat(DeleteFileSet.create()).isEmpty(); + assertThat(DeleteFileSet.create()) + .doesNotContain(FILE_A_DELETES, FILE_B_DELETES, FILE_C_DELETES); + } + + @Test + public void insertionOrderIsMaintained() { + DeleteFileSet set = DeleteFileSet.create(); + set.addAll(ImmutableList.of(FILE_D_DELETES, FILE_A_DELETES, FILE_C_DELETES)); + set.add(FILE_B_DELETES); + set.add(FILE_D_DELETES); + + assertThat(set) + .hasSize(4) + .containsExactly(FILE_D_DELETES, FILE_A_DELETES, FILE_C_DELETES, FILE_B_DELETES); + } + + @Test + public void clear() { + DeleteFileSet set = DeleteFileSet.of(ImmutableList.of(FILE_A_DELETES, FILE_B_DELETES)); + set.clear(); + assertThat(set).isEmpty(); + } + + @Test + public void addAll() { + DeleteFileSet empty = DeleteFileSet.create(); + assertThatThrownBy(() -> empty.add(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThatThrownBy(() -> empty.addAll(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid collection: null"); + + assertThatThrownBy(() -> empty.addAll(Collections.singletonList(null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThatThrownBy(() -> empty.addAll(Arrays.asList(FILE_A_DELETES, null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + DeleteFileSet set = DeleteFileSet.create(); + set.addAll(ImmutableList.of(FILE_B_DELETES, FILE_A_DELETES, FILE_C_DELETES, FILE_A_DELETES)); + assertThat(set).hasSize(3).containsExactly(FILE_B_DELETES, FILE_A_DELETES, FILE_C_DELETES); + } + + @Test + public void contains() { + DeleteFileSet set = DeleteFileSet.of(ImmutableList.of(FILE_A_DELETES, FILE_B_DELETES)); + assertThatThrownBy(() -> set.contains(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThat(set) + .hasSize(2) + .containsExactly(FILE_A_DELETES, FILE_B_DELETES) + .doesNotContain(FILE_C_DELETES) + .doesNotContain(FILE_D_DELETES); + + assertThatThrownBy( + () -> + DeleteFileSet.of( + Arrays.asList(FILE_C_DELETES, FILE_B_DELETES, null, FILE_A_DELETES))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + } + + @Test + public void containsAll() { + DeleteFileSet set = DeleteFileSet.of(ImmutableList.of(FILE_A_DELETES, FILE_B_DELETES)); + assertThatThrownBy(() -> set.containsAll(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid collection: null"); + + assertThatThrownBy(() -> set.containsAll(Collections.singletonList(null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThatThrownBy(() -> set.containsAll(Arrays.asList(FILE_A_DELETES, null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThat(set.containsAll(ImmutableList.of(FILE_B_DELETES, FILE_A_DELETES))).isTrue(); + assertThat(set.containsAll(ImmutableList.of(FILE_B_DELETES, FILE_A_DELETES, FILE_C_DELETES))) + .isFalse(); + assertThat(set.containsAll(ImmutableList.of(FILE_B_DELETES))).isTrue(); + } + + @Test + public void toArray() { + DeleteFileSet set = DeleteFileSet.of(ImmutableList.of(FILE_B_DELETES, FILE_A_DELETES)); + assertThat(set.toArray()).hasSize(2).containsExactly(FILE_B_DELETES, FILE_A_DELETES); + + DeleteFile[] array = new DeleteFile[1]; + assertThat(set.toArray(array)).hasSize(2).containsExactly(FILE_B_DELETES, FILE_A_DELETES); + + array = new DeleteFile[0]; + assertThat(set.toArray(array)).hasSize(2).containsExactly(FILE_B_DELETES, FILE_A_DELETES); + + array = new DeleteFile[5]; + assertThat(set.toArray(array)) + .hasSize(5) + .containsExactly(FILE_B_DELETES, FILE_A_DELETES, null, null, null); + + array = new DeleteFile[2]; + assertThat(set.toArray(array)).hasSize(2).containsExactly(FILE_B_DELETES, FILE_A_DELETES); + } + + @Test + public void retainAll() { + DeleteFileSet empty = DeleteFileSet.create(); + assertThatThrownBy(() -> empty.retainAll(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid collection: null"); + + assertThatThrownBy(() -> empty.retainAll(Collections.singletonList(null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThatThrownBy(() -> empty.retainAll(Arrays.asList(FILE_A_DELETES, null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + DeleteFileSet set = DeleteFileSet.of(ImmutableList.of(FILE_A_DELETES, FILE_B_DELETES)); + assertThat(set.retainAll(ImmutableList.of(FILE_C_DELETES, FILE_D_DELETES, FILE_A_DELETES))) + .as("Set should have changed") + .isTrue(); + + assertThat(set).hasSize(1).containsExactly(FILE_A_DELETES); + + set = DeleteFileSet.of(ImmutableList.of(FILE_A_DELETES, FILE_B_DELETES)); + + assertThat(set.retainAll(ImmutableList.of(FILE_B_DELETES, FILE_A_DELETES))) + .as("Set should not have changed") + .isFalse(); + + assertThat(set.retainAll(ImmutableList.of(FILE_C_DELETES, FILE_D_DELETES))) + .as("Set should have changed") + .isTrue(); + + assertThat(set).isEmpty(); + } + + @Test + public void remove() { + DeleteFileSet set = DeleteFileSet.of(ImmutableList.of(FILE_A_DELETES, FILE_B_DELETES)); + assertThatThrownBy(() -> set.remove(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + set.remove(FILE_C_DELETES); + assertThat(set).containsExactly(FILE_A_DELETES, FILE_B_DELETES); + assertThat(set).containsExactly(FILE_A_DELETES, FILE_B_DELETES); + set.remove(FILE_B_DELETES); + assertThat(set).containsExactly(FILE_A_DELETES); + set.remove(FILE_A_DELETES); + assertThat(set).isEmpty(); + } + + @Test + public void removeAll() { + DeleteFileSet empty = DeleteFileSet.create(); + assertThatThrownBy(() -> empty.removeAll(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid collection: null"); + + assertThatThrownBy(() -> empty.removeAll(Collections.singletonList(null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThatThrownBy(() -> empty.removeAll(Arrays.asList(FILE_A_DELETES, null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + DeleteFileSet set = DeleteFileSet.of(ImmutableList.of(FILE_A_DELETES, FILE_B_DELETES)); + assertThat(set.removeAll(ImmutableList.of(FILE_C_DELETES, FILE_D_DELETES, FILE_A_DELETES))) + .as("Set should have changed") + .isTrue(); + + assertThat(set).hasSize(1).containsExactly(FILE_B_DELETES); + + set = DeleteFileSet.of(ImmutableList.of(FILE_A_DELETES, FILE_B_DELETES)); + assertThat(set.removeAll(ImmutableList.of(FILE_C_DELETES, FILE_D_DELETES))) + .as("Set should not have changed") + .isFalse(); + + assertThat(set.removeAll(ImmutableList.of(FILE_B_DELETES, FILE_A_DELETES))) + .as("Set should have changed") + .isTrue(); + + assertThat(set).isEmpty(); + } + + @Test + public void equalsAndHashCode() { + DeleteFileSet set1 = DeleteFileSet.create(); + DeleteFileSet set2 = DeleteFileSet.create(); + + assertThat(set1).isEqualTo(set2); + assertThat(set1.hashCode()).isEqualTo(set2.hashCode()); + + set1.add(FILE_A_DELETES); + set1.add(FILE_B_DELETES); + set1.add(FILE_C_DELETES); + + // different DeleteFile instances but all use the same paths as set1 + set2.add( + FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofPositionDeletes() + .withPath(FILE_A_DELETES.location()) + .withFileSizeInBytes(10) + .withRecordCount(1) + .build()); + set2.add( + FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofPositionDeletes() + .withPath(FILE_B_DELETES.location()) + .withFileSizeInBytes(100) + .withRecordCount(10) + .build()); + set2.add( + FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofPositionDeletes() + .withPath(FILE_C_DELETES.location()) + .withFileSizeInBytes(1000) + .withRecordCount(100) + .build()); + + Set set3 = Collections.unmodifiableSet(set2); + + assertThat(set1).isEqualTo(set2).isEqualTo(set3); + assertThat(set1.hashCode()).isEqualTo(set2.hashCode()).isEqualTo(set3.hashCode()); + } + + @Test + public void kryoSerialization() throws Exception { + DeleteFileSet deleteFiles = + DeleteFileSet.of(ImmutableList.of(FILE_C_DELETES, FILE_B_DELETES, FILE_A_DELETES)); + assertThat(TestHelpers.KryoHelpers.roundTripSerialize(deleteFiles)).isEqualTo(deleteFiles); + } + + @Test + public void javaSerialization() throws Exception { + DeleteFileSet deleteFiles = + DeleteFileSet.of(ImmutableList.of(FILE_C_DELETES, FILE_B_DELETES, FILE_A_DELETES)); + DeleteFileSet deserialize = TestHelpers.deserialize(TestHelpers.serialize(deleteFiles)); + assertThat(deserialize).isEqualTo(deleteFiles); + } +} diff --git a/core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java b/core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java index c259bbd0a7e2..5e37e0390db9 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java +++ b/core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java @@ -20,7 +20,6 @@ import static org.assertj.core.api.Assertions.assertThat; -import java.io.IOException; import java.util.Collections; import java.util.Iterator; import java.util.List; @@ -43,8 +42,7 @@ public class TestParallelIterable { @Test - public void closeParallelIteratorWithoutCompleteIteration() - throws IOException, IllegalAccessException, NoSuchFieldException { + public void closeParallelIteratorWithoutCompleteIteration() { ExecutorService executor = Executors.newFixedThreadPool(1); Iterable> transform = @@ -76,8 +74,7 @@ public CloseableIterator iterator() { } @Test - public void closeMoreDataParallelIteratorWithoutCompleteIteration() - throws IOException, IllegalAccessException, NoSuchFieldException { + public void closeMoreDataParallelIteratorWithoutCompleteIteration() { ExecutorService executor = Executors.newFixedThreadPool(1); Iterator integerIterator = new Iterator() { @@ -137,8 +134,7 @@ public CloseableIterator iterator() { } @Test - public void limitQueueSize() throws IOException, IllegalAccessException, NoSuchFieldException { - + public void limitQueueSize() { List> iterables = ImmutableList.of( () -> IntStream.range(0, 100).iterator(), @@ -174,6 +170,41 @@ public void limitQueueSize() throws IOException, IllegalAccessException, NoSuchF executor.shutdownNow(); } + @Test + public void queueSizeOne() { + List> iterables = + ImmutableList.of( + () -> IntStream.range(0, 100).iterator(), + () -> IntStream.range(0, 100).iterator(), + () -> IntStream.range(0, 100).iterator()); + + Multiset expectedValues = + IntStream.range(0, 100) + .boxed() + .flatMap(i -> Stream.of(i, i, i)) + .collect(ImmutableMultiset.toImmutableMultiset()); + + ExecutorService executor = Executors.newCachedThreadPool(); + ParallelIterable parallelIterable = new ParallelIterable<>(iterables, executor, 1); + ParallelIterator iterator = (ParallelIterator) parallelIterable.iterator(); + + Multiset actualValues = HashMultiset.create(); + + while (iterator.hasNext()) { + assertThat(iterator.queueSize()) + .as("iterator internal queue size") + .isLessThanOrEqualTo(1 + iterables.size()); + actualValues.add(iterator.next()); + } + + assertThat(actualValues) + .as("multiset of values returned by the iterator") + .isEqualTo(expectedValues); + + iterator.close(); + executor.shutdownNow(); + } + private void queueHasElements(ParallelIterator iterator) { assertThat(iterator.hasNext()).isTrue(); assertThat(iterator.next()).isNotNull(); diff --git a/core/src/test/java/org/apache/iceberg/view/TestViewVersionParser.java b/core/src/test/java/org/apache/iceberg/view/TestViewVersionParser.java index 1d115c04c393..a68b99a6797b 100644 --- a/core/src/test/java/org/apache/iceberg/view/TestViewVersionParser.java +++ b/core/src/test/java/org/apache/iceberg/view/TestViewVersionParser.java @@ -129,4 +129,22 @@ public void missingDefaultCatalog() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing field: default-namespace"); } + + @Test + public void invalidRepresentations() { + String invalidRepresentations = + "{\"version-id\":1, \"timestamp-ms\":12345, \"schema-id\":1, \"summary\":{\"user\":\"some-user\"}, \"representations\": 23, \"default-namespace\":[\"one\",\"two\"]}"; + assertThatThrownBy(() -> ViewVersionParser.fromJson(invalidRepresentations)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse representations from non-array: 23"); + } + + @Test + public void missingRepresentations() { + String missingRepresentations = + "{\"version-id\":1, \"timestamp-ms\":12345, \"schema-id\":1, \"summary\":{\"user\":\"some-user\"}, \"default-namespace\":[\"one\",\"two\"]}"; + assertThatThrownBy(() -> ViewVersionParser.fromJson(missingRepresentations)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing field: representations"); + } } diff --git a/core/src/test/resources/org/apache/iceberg/deletes/64map32bitvals.bin b/core/src/test/resources/org/apache/iceberg/deletes/64map32bitvals.bin new file mode 100644 index 000000000000..475b894417e4 Binary files /dev/null and b/core/src/test/resources/org/apache/iceberg/deletes/64map32bitvals.bin differ diff --git a/core/src/test/resources/org/apache/iceberg/deletes/64mapempty.bin b/core/src/test/resources/org/apache/iceberg/deletes/64mapempty.bin new file mode 100644 index 000000000000..1b1cb4d44c57 Binary files /dev/null and b/core/src/test/resources/org/apache/iceberg/deletes/64mapempty.bin differ diff --git a/core/src/test/resources/org/apache/iceberg/deletes/64maphighvals.bin b/core/src/test/resources/org/apache/iceberg/deletes/64maphighvals.bin new file mode 100644 index 000000000000..d4312b8d2271 Binary files /dev/null and b/core/src/test/resources/org/apache/iceberg/deletes/64maphighvals.bin differ diff --git a/core/src/test/resources/org/apache/iceberg/deletes/64mapspreadvals.bin b/core/src/test/resources/org/apache/iceberg/deletes/64mapspreadvals.bin new file mode 100644 index 000000000000..83c72f6ba327 Binary files /dev/null and b/core/src/test/resources/org/apache/iceberg/deletes/64mapspreadvals.bin differ diff --git a/data/src/main/java/org/apache/iceberg/data/BaseDeleteLoader.java b/data/src/main/java/org/apache/iceberg/data/BaseDeleteLoader.java index 91b7fd1c1dc1..8a1ebf95abeb 100644 --- a/data/src/main/java/org/apache/iceberg/data/BaseDeleteLoader.java +++ b/data/src/main/java/org/apache/iceberg/data/BaseDeleteLoader.java @@ -166,13 +166,13 @@ private PositionDeleteIndex getOrReadPosDeletes(DeleteFile deleteFile, CharSeque private CharSequenceMap readPosDeletes(DeleteFile deleteFile) { CloseableIterable deletes = openDeletes(deleteFile, POS_DELETE_SCHEMA); - return Deletes.toPositionIndexes(deletes); + return Deletes.toPositionIndexes(deletes, deleteFile); } private PositionDeleteIndex readPosDeletes(DeleteFile deleteFile, CharSequence filePath) { Expression filter = Expressions.equal(MetadataColumns.DELETE_FILE_PATH.name(), filePath); CloseableIterable deletes = openDeletes(deleteFile, POS_DELETE_SCHEMA, filter); - return Deletes.toPositionIndex(filePath, ImmutableList.of(deletes)); + return Deletes.toPositionIndex(filePath, deletes, deleteFile); } private CloseableIterable openDeletes(DeleteFile deleteFile, Schema projection) { diff --git a/data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java b/data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java index da6f2bfd1152..0602c9e494c7 100644 --- a/data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java +++ b/data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java @@ -97,7 +97,9 @@ public static List listPartition( * @param conf a Hadoop conf * @param metricsSpec a metrics conf * @param mapping a name mapping - * @param parallelism number of threads to use for file reading + * @param parallelism number of threads to use for file reading. If null, file reading will be + * performed on the current thread. If non-null, the provided ExecutorService will be shutdown + * within this method after file reading is complete. * @return a List of DataFile */ public static List listPartition( @@ -137,7 +139,9 @@ public static List listPartition( * @param conf a Hadoop conf * @param metricsSpec a metrics conf * @param mapping a name mapping - * @param service executor service to use for file reading + * @param service executor service to use for file reading. If null, file reading will be + * performed on the current thread. If non-null, the provided ExecutorService will be shutdown + * within this method after file reading is complete. * @return a List of DataFile */ public static List listPartition( @@ -250,7 +254,16 @@ private static DataFile buildDataFile( .build(); } + /** + * Returns an {@link ExecutorService} for table migration. + * + *

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

Important: Callers are responsible for shutting down the returned executor service + * when it is no longer needed to prevent resource leaks. + */ public static ExecutorService migrationService(int parallelism) { - return parallelism == 1 ? null : ThreadPools.newWorkerPool("table-migration", parallelism); + return parallelism == 1 ? null : ThreadPools.newFixedThreadPool("table-migration", parallelism); } } diff --git a/data/src/test/java/org/apache/iceberg/io/TestPartitioningWriters.java b/data/src/test/java/org/apache/iceberg/io/TestPartitioningWriters.java index 8dc031314eda..1c8453bd6a75 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestPartitioningWriters.java +++ b/data/src/test/java/org/apache/iceberg/io/TestPartitioningWriters.java @@ -20,13 +20,17 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; import java.io.IOException; import java.nio.file.Files; import java.util.Arrays; import java.util.List; +import java.util.Map; +import java.util.function.Function; import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Parameter; import org.apache.iceberg.ParameterizedTestExtension; @@ -34,10 +38,16 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.RowDelta; import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.BaseDeleteLoader; +import org.apache.iceberg.data.DeleteLoader; import org.apache.iceberg.deletes.DeleteGranularity; import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.deletes.PositionDeleteIndex; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.ContentFileUtil; import org.apache.iceberg.util.StructLikeSet; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -719,4 +729,105 @@ private void checkFanoutPositionOnlyDeleteWriterGranularity(DeleteGranularity de List expectedRows = ImmutableList.of(toRow(11, "aaa"), toRow(12, "aaa")); assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows)); } + + @TestTemplate + public void testRewriteOfPreviousDeletes() throws IOException { + assumeThat(format()).isIn(FileFormat.PARQUET, FileFormat.ORC); + + FileWriterFactory writerFactory = newWriterFactory(table.schema()); + + // add the first data file + List rows1 = ImmutableList.of(toRow(1, "aaa"), toRow(2, "aaa"), toRow(11, "aaa")); + DataFile dataFile1 = writeData(writerFactory, fileFactory, rows1, table.spec(), null); + table.newFastAppend().appendFile(dataFile1).commit(); + + // add the second data file + List rows2 = ImmutableList.of(toRow(3, "aaa"), toRow(4, "aaa"), toRow(12, "aaa")); + DataFile dataFile2 = writeData(writerFactory, fileFactory, rows2, table.spec(), null); + table.newFastAppend().appendFile(dataFile2).commit(); + + PartitionSpec spec = table.spec(); + + // init the first delete writer without access to previous deletes + FanoutPositionOnlyDeleteWriter writer1 = + new FanoutPositionOnlyDeleteWriter<>( + writerFactory, fileFactory, table.io(), TARGET_FILE_SIZE, DeleteGranularity.FILE); + + // write initial deletes for both data files + writer1.write(positionDelete(dataFile1.path(), 1L), spec, null); + writer1.write(positionDelete(dataFile2.path(), 1L), spec, null); + writer1.close(); + + // verify the writer result + DeleteWriteResult result1 = writer1.result(); + assertThat(result1.deleteFiles()).hasSize(2); + assertThat(result1.referencedDataFiles()).hasSize(2); + assertThat(result1.referencesDataFiles()).isTrue(); + assertThat(result1.rewrittenDeleteFiles()).isEmpty(); + + // commit the initial deletes + RowDelta rowDelta1 = table.newRowDelta(); + result1.deleteFiles().forEach(rowDelta1::addDeletes); + rowDelta1.commit(); + + // verify correctness of the first delete operation + List expectedRows1 = + ImmutableList.of(toRow(1, "aaa"), toRow(3, "aaa"), toRow(11, "aaa"), toRow(12, "aaa")); + assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows1)); + + // populate previous delete mapping + Map previousDeletes = Maps.newHashMap(); + for (DeleteFile deleteFile : result1.deleteFiles()) { + String dataLocation = ContentFileUtil.referencedDataFile(deleteFile).toString(); + previousDeletes.put(dataLocation, deleteFile); + } + + // init the second delete writer with access to previous deletes + FanoutPositionOnlyDeleteWriter writer2 = + new FanoutPositionOnlyDeleteWriter<>( + writerFactory, + fileFactory, + table.io(), + TARGET_FILE_SIZE, + DeleteGranularity.FILE, + new PreviousDeleteLoader(table, previousDeletes)); + + // write more deletes for both data files + writer2.write(positionDelete(dataFile1.path(), 0L), spec, null); + writer2.write(positionDelete(dataFile2.path(), 0L), spec, null); + writer2.close(); + + // verify the writer result + DeleteWriteResult result2 = writer2.result(); + assertThat(result2.deleteFiles()).hasSize(2); + assertThat(result2.referencedDataFiles()).hasSize(2); + assertThat(result2.referencesDataFiles()).isTrue(); + assertThat(result2.rewrittenDeleteFiles()).hasSize(2); + + // add new and remove rewritten delete files + RowDelta rowDelta2 = table.newRowDelta(); + result2.deleteFiles().forEach(rowDelta2::addDeletes); + result2.rewrittenDeleteFiles().forEach(rowDelta2::removeDeletes); + rowDelta2.commit(); + + // verify correctness of the second delete operation + List expectedRows2 = ImmutableList.of(toRow(11, "aaa"), toRow(12, "aaa")); + assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows2)); + } + + private static class PreviousDeleteLoader implements Function { + private final Map deleteFiles; + private final DeleteLoader deleteLoader; + + PreviousDeleteLoader(Table table, Map deleteFiles) { + this.deleteFiles = deleteFiles; + this.deleteLoader = new BaseDeleteLoader(deleteFile -> table.io().newInputFile(deleteFile)); + } + + @Override + public PositionDeleteIndex apply(CharSequence path) { + DeleteFile deleteFile = deleteFiles.get(path); + return deleteLoader.loadPositionDeletes(ImmutableList.of(deleteFile), path); + } + } } diff --git a/dell/src/main/java/org/apache/iceberg/dell/ecs/EcsSeekableInputStream.java b/dell/src/main/java/org/apache/iceberg/dell/ecs/EcsSeekableInputStream.java index e59b03ae44bc..6d32d3ba981d 100644 --- a/dell/src/main/java/org/apache/iceberg/dell/ecs/EcsSeekableInputStream.java +++ b/dell/src/main/java/org/apache/iceberg/dell/ecs/EcsSeekableInputStream.java @@ -46,6 +46,7 @@ class EcsSeekableInputStream extends SeekableInputStream { /** Mutable pos set by {@link #seek(long)} */ private long newPos = 0; + /** Current pos of object content */ private long pos = -1; diff --git a/deploy.gradle b/deploy.gradle index 462448303253..37a6262f1efe 100644 --- a/deploy.gradle +++ b/deploy.gradle @@ -22,12 +22,8 @@ if (project.hasProperty('release') && jdkVersion != '11') { } subprojects { - if (it.name == 'iceberg-open-api') { - // don't publish iceberg-open-api - return - } - def isBom = it.name == 'iceberg-bom' + def isOpenApi = it.name == 'iceberg-open-api' apply plugin: 'maven-publish' apply plugin: 'signing' @@ -76,6 +72,9 @@ subprojects { apache(MavenPublication) { if (isBom) { from components.javaPlatform + } else if (isOpenApi) { + artifact testJar + artifact testFixturesJar } else { if (tasks.matching({task -> task.name == 'shadowJar'}).isEmpty()) { from components.java diff --git a/dev/stage-binaries.sh b/dev/stage-binaries.sh index 29cf31e5f423..fa09b76c38e9 100755 --- a/dev/stage-binaries.sh +++ b/dev/stage-binaries.sh @@ -22,8 +22,9 @@ SCALA_VERSION=2.12 FLINK_VERSIONS=1.18,1.19,1.20 SPARK_VERSIONS=3.3,3.4,3.5 HIVE_VERSIONS=2,3 +KAFKA_VERSIONS=3 -./gradlew -Prelease -DscalaVersion=$SCALA_VERSION -DflinkVersions=$FLINK_VERSIONS -DsparkVersions=$SPARK_VERSIONS -DhiveVersions=$HIVE_VERSIONS publishApachePublicationToMavenRepository +./gradlew -Prelease -DscalaVersion=$SCALA_VERSION -DflinkVersions=$FLINK_VERSIONS -DsparkVersions=$SPARK_VERSIONS -DhiveVersions=$HIVE_VERSIONS -DkafkaVersions=$KAFKA_VERSIONS publishApachePublicationToMavenRepository # Also publish Scala 2.13 Artifacts for versions that support it. # Flink does not yet support 2.13 (and is largely dropping a user-facing dependency on Scala). Hive doesn't need a Scala specification. diff --git a/doap.rdf b/doap.rdf index 4f2bd4222c1e..51896c6a8037 100644 --- a/doap.rdf +++ b/doap.rdf @@ -41,9 +41,9 @@ - 1.6.0 - 2024-07-23 - 1.6.0 + 1.6.1 + 2024-08-27 + 1.6.1 diff --git a/docs/docs/aws.md b/docs/docs/aws.md index 43e540c20673..1a98a4d18e5b 100644 --- a/docs/docs/aws.md +++ b/docs/docs/aws.md @@ -343,7 +343,10 @@ Data stored in S3 with a traditional Hive storage layout can face S3 request thr Iceberg by default uses the Hive storage layout but can be switched to use the `ObjectStoreLocationProvider`. With `ObjectStoreLocationProvider`, a deterministic hash is generated for each stored file, with the hash appended -directly after the `write.data.path`. This ensures files written to s3 are equally distributed across multiple [prefixes](https://aws.amazon.com/premiumsupport/knowledge-center/s3-object-key-naming-pattern/) in the S3 bucket. Resulting in minimized throttling and maximized throughput for S3-related IO operations. When using `ObjectStoreLocationProvider` having a shared and short `write.data.path` across your Iceberg tables will improve performance. +directly after the `write.data.path`. This ensures files written to S3 are equally distributed across multiple +[prefixes](https://aws.amazon.com/premiumsupport/knowledge-center/s3-object-key-naming-pattern/) in the S3 bucket; +resulting in minimized throttling and maximized throughput for S3-related IO operations. When using `ObjectStoreLocationProvider` +having a shared `write.data.path` across your Iceberg tables will improve performance. For more information on how S3 scales API QPS, check out the 2018 re:Invent session on [Best Practices for Amazon S3 and Amazon S3 Glacier](https://youtu.be/rHeTn9pHNKo?t=3219). At [53:39](https://youtu.be/rHeTn9pHNKo?t=3219) it covers how S3 scales/partitions & at [54:50](https://youtu.be/rHeTn9pHNKo?t=3290) it discusses the 30-60 minute wait time before new partitions are created. @@ -357,7 +360,7 @@ CREATE TABLE my_catalog.my_ns.my_table ( USING iceberg OPTIONS ( 'write.object-storage.enabled'=true, - 'write.data.path'='s3://my-table-data-bucket') + 'write.data.path'='s3://my-table-data-bucket/my_table') PARTITIONED BY (category); ``` @@ -366,9 +369,16 @@ We can then insert a single row into this new table INSERT INTO my_catalog.my_ns.my_table VALUES (1, "Pizza", "orders"); ``` -Which will write the data to S3 with a hash (`2d3905f8`) appended directly after the `write.object-storage.path`, ensuring reads to the table are spread evenly across [S3 bucket prefixes](https://docs.aws.amazon.com/AmazonS3/latest/userguide/optimizing-performance.html), and improving performance. +Which will write the data to S3 with a 20-bit base2 hash (`01010110100110110010`) appended directly after the `write.object-storage.path`, +ensuring reads to the table are spread evenly across [S3 bucket prefixes](https://docs.aws.amazon.com/AmazonS3/latest/userguide/optimizing-performance.html), and improving performance. +Previously provided base64 hash was updated to base2 in order to provide an improved auto-scaling behavior on S3 General Purpose Buckets. + +As part of this update, we have also divided the entropy into multiple directories in order to improve the efficiency of the +orphan clean up process for Iceberg since directories are used as a mean to divide the work across workers for faster traversal. You +can see from the example below that we divide the hash to create 4-bit directories with a depth of 3 and attach the final part of the hash to +the end. ``` -s3://my-table-data-bucket/2d3905f8/my_ns.db/my_table/category=orders/00000-0-5affc076-96a4-48f2-9cd2-d5efbc9f0c94-00001.parquet +s3://my-table-data-bucket/my_ns.db/my_table/0101/0110/1001/10110010/category=orders/00000-0-5affc076-96a4-48f2-9cd2-d5efbc9f0c94-00001.parquet ``` Note, the path resolution logic for `ObjectStoreLocationProvider` is `write.data.path` then `/data`. @@ -378,6 +388,28 @@ However, for the older versions up to 0.12.0, the logic is as follows: For more details, please refer to the [LocationProvider Configuration](custom-catalog.md#custom-location-provider-implementation) section. +We have also added a new table property `write.object-storage.partitioned-paths` that if set to false(default=true), this will +omit the partition values from the file path. Iceberg does not need these values in the file path and setting this value to false +can further reduce the key size. In this case, we also append the final 8 bit of entropy directly to the file name. +Inserted key would look like the following with this config set, note that `category=orders` is removed: +``` +s3://my-table-data-bucket/my_ns.db/my_table/1101/0100/1011/00111010-00000-0-5affc076-96a4-48f2-9cd2-d5efbc9f0c94-00001.parquet +``` + +### S3 Retries + +Workloads which encounter S3 throttling should persistently retry, with exponential backoff, to make progress while S3 +automatically scales. We provide the configurations below to adjust S3 retries for this purpose. For workloads that encounter +throttling and fail due to retry exhaustion, we recommend retry count to set 32 in order allow S3 to auto-scale. Note that +workloads with exceptionally high throughput against tables that S3 has not yet scaled, it may be necessary to increase the retry count further. + + +| Property | Default | Description | +|----------------------|---------|---------------------------------------------------------------------------------------| +| s3.retry.num-retries | 5 | Number of times to retry S3 operations. Recommended 32 for high-throughput workloads. | +| s3.retry.min-wait-ms | 2s | Minimum wait time to retry a S3 operation. | +| s3.retry.max-wait-ms | 20s | Maximum wait time to retry a S3 read operation. | + ### S3 Strong Consistency In November 2020, S3 announced [strong consistency](https://aws.amazon.com/s3/consistency/) for all read operations, and Iceberg is updated to fully leverage this feature. @@ -468,13 +500,13 @@ spark-sql --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCata --conf spark.sql.catalog.my_catalog.type=glue \ --conf spark.sql.catalog.my_catalog.io-impl=org.apache.iceberg.aws.s3.S3FileIO \ --conf spark.sql.catalog.my_catalog.s3.use-arn-region-enabled=false \ - --conf spark.sql.catalog.my_catalog.s3.access-points.my-bucket1=arn:aws:s3::123456789012:accesspoint:mfzwi23gnjvgw.mrap \ - --conf spark.sql.catalog.my_catalog.s3.access-points.my-bucket2=arn:aws:s3::123456789012:accesspoint:mfzwi23gnjvgw.mrap + --conf spark.sql.catalog.my_catalog.s3.access-points.my-bucket1=arn:aws:s3:::accesspoint/ \ + --conf spark.sql.catalog.my_catalog.s3.access-points.my-bucket2=arn:aws:s3:::accesspoint/ ``` -For the above example, the objects in S3 on `my-bucket1` and `my-bucket2` buckets will use `arn:aws:s3::123456789012:accesspoint:mfzwi23gnjvgw.mrap` +For the above example, the objects in S3 on `my-bucket1` and `my-bucket2` buckets will use `arn:aws:s3:::accesspoint/` access-point for all S3 operations. -For more details on using access-points, please refer [Using access points with compatible Amazon S3 operations](https://docs.aws.amazon.com/AmazonS3/latest/userguide/access-points-usage-examples.html). +For more details on using access-points, please refer [Using access points with compatible Amazon S3 operations](https://docs.aws.amazon.com/AmazonS3/latest/userguide/access-points-usage-examples.html), [Sample notebook](https://github.com/aws-samples/quant-research/tree/main) . ### S3 Access Grants @@ -500,6 +532,22 @@ spark-sql --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCata For more details on using S3 Access Grants, please refer to [Managing access with S3 Access Grants](https://docs.aws.amazon.com/AmazonS3/latest/userguide/access-grants.html). +### S3 Cross-Region Access + +S3 Cross-Region bucket access can be turned on by setting catalog property `s3.cross-region-access-enabled` to `true`. +This is turned off by default to avoid first S3 API call increased latency. + +For example, to enable S3 Cross-Region bucket access with Spark 3.3, you can start the Spark SQL shell with: +``` +spark-sql --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \ + --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket2/my/key/prefix \ + --conf spark.sql.catalog.my_catalog.type=glue \ + --conf spark.sql.catalog.my_catalog.io-impl=org.apache.iceberg.aws.s3.S3FileIO \ + --conf spark.sql.catalog.my_catalog.s3.cross-region-access-enabled=true +``` + +For more details, please refer to [Cross-Region access for Amazon S3](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/s3-cross-region.html). + ### S3 Acceleration [S3 Acceleration](https://aws.amazon.com/s3/transfer-acceleration/) can be used to speed up transfers to and from Amazon S3 by as much as 50-500% for long-distance transfer of larger objects. @@ -685,3 +733,9 @@ Search the [Iceberg blogs](../../blogs.md) page for tutorials around running Ice [Amazon Kinesis Data Analytics](https://aws.amazon.com/about-aws/whats-new/2019/11/you-can-now-run-fully-managed-apache-flink-applications-with-apache-kafka/) provides a platform to run fully managed Apache Flink applications. You can include Iceberg in your application Jar and run it in the platform. + +### AWS Redshift +[AWS Redshift Spectrum or Redshift Serverless](https://docs.aws.amazon.com/redshift/latest/dg/querying-iceberg.html) supports querying Apache Iceberg tables cataloged in the AWS Glue Data Catalog. + +### Amazon Data Firehose +You can use [Firehose](https://docs.aws.amazon.com/firehose/latest/dev/apache-iceberg-destination.html) to directly deliver streaming data to Apache Iceberg Tables in Amazon S3. With this feature, you can route records from a single stream into different Apache Iceberg Tables, and automatically apply insert, update, and delete operations to records in the Apache Iceberg Tables. This feature requires using the AWS Glue Data Catalog. \ No newline at end of file diff --git a/docs/docs/branching.md b/docs/docs/branching.md index f00defda665b..bfb976bfa206 100644 --- a/docs/docs/branching.md +++ b/docs/docs/branching.md @@ -127,7 +127,7 @@ Create a table and insert some data: ```sql CREATE TABLE db.table (id bigint, data string, col float); -INSERT INTO db.table values (1, 'a', 1.0), (2, 'b', 2.0), (3, 'c', 3.0); +INSERT INTO db.table VALUES (1, 'a', 1.0), (2, 'b', 2.0), (3, 'c', 3.0); SELECT * FROM db.table; 1 a 1.0 2 b 2.0 @@ -148,11 +148,11 @@ SELECT * FROM db.table.branch_test_branch; Modify the table's schema by dropping the `col` column and adding a new column named `new_col`: ```sql -ALTER TABLE db.table drop column col; +ALTER TABLE db.table DROP COLUMN col; -ALTER TABLE db.table add column new_col date; +ALTER TABLE db.table ADD COLUMN new_col date; -INSERT INTO db.table values (4, 'd', date('2024-04-04')), (5, 'e', date('2024-05-05')); +INSERT INTO db.table VALUES (4, 'd', date('2024-04-04')), (5, 'e', date('2024-05-05')); SELECT * FROM db.table; 1 a NULL @@ -195,7 +195,7 @@ When writing to the branch, the **table's schema** is used for validation: ```sql -INSERT INTO db.table.branch_test_branch values (6, 'e', date('2024-06-06')), (7, 'g', date('2024-07-07')); +INSERT INTO db.table.branch_test_branch VALUES (6, 'e', date('2024-06-06')), (7, 'g', date('2024-07-07')); SELECT * FROM db.table.branch_test_branch; 6 e 2024-06-06 diff --git a/docs/docs/configuration.md b/docs/docs/configuration.md index 264b9edfa7cc..07a98fd94515 100644 --- a/docs/docs/configuration.md +++ b/docs/docs/configuration.md @@ -77,6 +77,7 @@ Iceberg tables support table properties to configure table behavior, like the de | write.metadata.previous-versions-max | 100 | The max number of previous version metadata files to keep before deleting after commit | | write.spark.fanout.enabled | false | Enables the fanout writer in Spark that does not require data to be clustered; uses more memory | | write.object-storage.enabled | false | Enables the object storage location provider that adds a hash component to file paths | +| write.object-storage.partitioned-paths | true | Includes the partition values in the file path | | write.data.path | table location + /data | Base location for data files | | write.metadata.path | table location + /metadata | Base location for metadata files | | write.delete.mode | copy-on-write | Mode used for delete commands: copy-on-write or merge-on-read (v2 only) | diff --git a/docs/docs/flink-configuration.md b/docs/docs/flink-configuration.md index 42dc15f5b3d2..1ac16d7fc3e9 100644 --- a/docs/docs/flink-configuration.md +++ b/docs/docs/flink-configuration.md @@ -146,14 +146,56 @@ INSERT INTO tableName /*+ OPTIONS('upsert-enabled'='true') */ ... ``` -| Flink option | Default | Description | -| ---------------------- | ------------------------------------------ | ------------------------------------------------------------ | -| write-format | Table write.format.default | File format to use for this write operation; parquet, avro, or orc | -| target-file-size-bytes | As per table property | Overrides this table's write.target-file-size-bytes | -| upsert-enabled | Table write.upsert.enabled | Overrides this table's write.upsert.enabled | -| overwrite-enabled | false | Overwrite the table's data, overwrite mode shouldn't be enable when configuring to use UPSERT data stream. | -| distribution-mode | Table write.distribution-mode | Overrides this table's write.distribution-mode | -| compression-codec | Table write.(fileformat).compression-codec | Overrides this table's compression codec for this write | -| compression-level | Table write.(fileformat).compression-level | Overrides this table's compression level for Parquet and Avro tables for this write | -| compression-strategy | Table write.orc.compression-strategy | Overrides this table's compression strategy for ORC tables for this write | -| write-parallelism | Upstream operator parallelism | Overrides the writer parallelism | +| Flink option | Default | Description | +|-----------------------------------------|--------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------| +| write-format | Table write.format.default | File format to use for this write operation; parquet, avro, or orc | +| target-file-size-bytes | As per table property | Overrides this table's write.target-file-size-bytes | +| upsert-enabled | Table write.upsert.enabled | Overrides this table's write.upsert.enabled | +| overwrite-enabled | false | Overwrite the table's data, overwrite mode shouldn't be enable when configuring to use UPSERT data stream. | +| distribution-mode | Table write.distribution-mode | Overrides this table's write.distribution-mode. RANGE distribution is in experimental status. | +| range-distribution-statistics-type | Auto | Range distribution data statistics collection type: Map, Sketch, Auto. See details [here](#range-distribution-statistics-type). | +| range-distribution-sort-key-base-weight | 0.0 (double) | Base weight for every sort key relative to target traffic weight per writer task. See details [here](#range-distribution-sort-key-base-weight). | +| compression-codec | Table write.(fileformat).compression-codec | Overrides this table's compression codec for this write | +| compression-level | Table write.(fileformat).compression-level | Overrides this table's compression level for Parquet and Avro tables for this write | +| compression-strategy | Table write.orc.compression-strategy | Overrides this table's compression strategy for ORC tables for this write | +| write-parallelism | Upstream operator parallelism | Overrides the writer parallelism | + +#### Range distribution statistics type + +Config value is a enum type: `Map`, `Sketch`, `Auto`. +

    +
  • Map: collects accurate sampling count for every single key. +It should be used for low cardinality scenarios (like hundreds or thousands). +
  • Sketch: constructs a uniform random sampling via reservoir sampling. +It fits well for high cardinality scenarios (like millions), as memory footprint is kept low. +
  • Auto: starts with Maps statistics. But if cardinality is detected higher +than a threshold (currently 10,000), statistics are automatically switched to Sketch. +
+ +#### Range distribution sort key base weight + +`range-distribution-sort-key-base-weight`: `0.0`. + +If sort order contains partition columns, each sort key would map to one partition and data +file. This relative weight can avoid placing too many small files for sort keys with low +traffic. It is a double value that defines the minimal weight for each sort key. `0.02` means +each key has a base weight of `2%` of the targeted traffic weight per writer task. + +E.g. the sink Iceberg table is partitioned daily by event time. Assume the data stream +contains events from now up to 180 days ago. With event time, traffic weight distribution +across different days typically has a long tail pattern. Current day contains the most +traffic. The older days (long tail) contain less and less traffic. Assume writer parallelism +is `10`. The total weight across all 180 days is `10,000`. Target traffic weight per writer +task would be `1,000`. Assume the weight sum for the oldest 150 days is `1,000`. Normally, +the range partitioner would put all the oldest 150 days in one writer task. That writer task +would write to 150 small files (one per day). Keeping 150 open files can potentially consume +large amount of memory. Flushing and uploading 150 files (however small) at checkpoint time +can also be potentially slow. If this config is set to `0.02`. It means every sort key has a +base weight of `2%` of targeted weight of `1,000` for every write task. It would essentially +avoid placing more than `50` data files (one per day) on one writer task no matter how small +they are. + +This is only applicable to {@link StatisticsType#Map} for low-cardinality scenario. For +{@link StatisticsType#Sketch} high-cardinality sort columns, they are usually not used as +partition columns. Otherwise, too many partitions and small files may be generated during +write. Sketch range partitioner simply splits high-cardinality keys into ordered ranges. \ No newline at end of file diff --git a/docs/docs/flink-queries.md b/docs/docs/flink-queries.md index 38263d47c199..53313db9cf5e 100644 --- a/docs/docs/flink-queries.md +++ b/docs/docs/flink-queries.md @@ -66,13 +66,17 @@ There are some options that could be set in Flink SQL hint options for streaming ### FLIP-27 source for SQL -Here are the SQL settings for the [FLIP-27](https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface) source. All other SQL settings and options documented above are applicable to the FLIP-27 source. +Here is the SQL setting to opt in or out of the +[FLIP-27 source](https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface). ```sql --- Opt in the FLIP-27 source. Default is false. -SET table.exec.iceberg.use-flip27-source = true; +-- Opt out the FLIP-27 source. +-- Default is false for Flink 1.19 and below, and true for Flink 1.20 and above. +SET table.exec.iceberg.use-flip27-source = false; ``` +All other SQL settings and options documented above are applicable to the FLIP-27 source. + ### Reading branches and tags with SQL Branch and tags can be read via SQL by specifying options. For more details refer to [Flink Configuration](flink-configuration.md#read-options) diff --git a/docs/docs/flink-writes.md b/docs/docs/flink-writes.md index b916a5f9b7b0..f53b5d832efe 100644 --- a/docs/docs/flink-writes.md +++ b/docs/docs/flink-writes.md @@ -262,6 +262,107 @@ INSERT INTO tableName /*+ OPTIONS('upsert-enabled'='true') */ Check out all the options here: [write-options](flink-configuration.md#write-options) +## Distribution mode + +Flink streaming writer supports both `HASH` and `RANGE` distribution mode. +You can enable it via `FlinkSink#Builder#distributionMode(DistributionMode)` +or via [write-options](flink-configuration.md#write-options). + +### Hash distribution + +HASH distribution shuffles data by partition key (partitioned table) or +equality fields (non-partitioned table). It simply leverages Flink's +`DataStream#keyBy` to distribute the data. + +HASH distribution has a few limitations. +
    +
  • It doesn't handle skewed data well. E.g. some partitions have a lot more data than others. +
  • It can result in unbalanced traffic distribution if cardinality of the partition key or +equality fields is low as demonstrated by [PR 4228](https://github.com/apache/iceberg/pull/4228). +
  • Writer parallelism is limited to the cardinality of the hash key. +If the cardinality is 10, only at most 10 writer tasks would get the traffic. +Having higher writer parallelism (even if traffic volume requires) won't help. +
+ +### Range distribution (experimental) + +RANGE distribution shuffles data by partition key or sort order via a custom range partitioner. +Range distribution collects traffic statistics to guide the range partitioner to +evenly distribute traffic to writer tasks. + +Range distribution only shuffle the data via range partitioner. Rows are *not* sorted within +a data file, which Flink streaming writer doesn't support yet. + +#### Use cases + +RANGE distribution can be applied to an Iceberg table that either is partitioned or +has SortOrder defined. For a partitioned table without SortOrder, partition columns +are used as sort order. If SortOrder is explicitly defined for the table, it is used by +the range partitioner. + +Range distribution can handle skewed data. E.g. +
    +
  • Table is partitioned by event time. Typically, recent hours have more data, +while the long-tail hours have less and less data. +
  • Table is partitioned by country code, where some countries (like US) have +a lot more traffic and smaller countries have a lot less data +
  • Table is partitioned by event type, where some types have a lot more data than others. +
+ +Range distribution can also cluster data on non-partition columns. +E.g., table is partitioned hourly on ingestion time. Queries often include +predicate on a non-partition column like `device_id` or `country_code`. +Range partition would improve the query performance by clustering on the non-partition column +when table `SortOrder` is defined with the non-partition column. + +#### Traffic statistics + +Statistics are collected by every shuffle operator subtask and aggregated by the coordinator +for every checkpoint cycle. Aggregated statistics are broadcast to all subtasks and +applied to the range partitioner in the next checkpoint. So it may take up to two checkpoint +cycles to detect traffic distribution change and apply the new statistics to range partitioner. + +Range distribution can work with low cardinality (like `country_code`) +or high cardinality (like `device_id`) scenarios. +
    +
  • For low cardinality scenario (like hundreds or thousands), +HashMap is used to track traffic distribution for every key. +If a new sort key value shows up, range partitioner would just +round-robin it to the writer tasks before traffic distribution has been learned +about the new key. +
  • For high cardinality scenario (like millions or billions), +uniform random sampling (reservoir sampling) is used to compute range bounds +that split the sort key space evenly. +It keeps the memory footprint and network exchange low. +Reservoir sampling work well if key distribution is relatively even. +If a single hot key has unbalanced large share of the traffic, +range split by uniform sampling probably won't work very well. +
+ +#### Usage + +Here is how to enable range distribution in Java. There are two optional advanced configs. Default should +work well for most cases. See [write-options](flink-configuration.md#write-options) for details. +```java +FlinkSink.forRowData(input) + ... + .distributionMode(DistributionMode.RANGE) + .rangeDistributionStatisticsType(StatisticsType.Auto) + .rangeDistributionSortKeyBaseWeight(0.0d) + .append(); +``` + +### Overhead + +Data shuffling (hash or range) has computational overhead of serialization/deserialization +and network I/O. Expect some increase of CPU utilization. + +Range distribution also collect and aggregate data distribution statistics. +That would also incur some CPU overhead. Memory overhead is typically +small if using default statistics type of `Auto`. Don't use `Map` statistics +type if key cardinality is high. That could result in significant memory footprint +and large network exchange for statistics aggregation. + ## Notes Flink streaming write jobs rely on snapshot summary to keep the last committed checkpoint ID, and diff --git a/docs/docs/kafka-connect.md b/docs/docs/kafka-connect.md new file mode 100644 index 000000000000..a904a17a9968 --- /dev/null +++ b/docs/docs/kafka-connect.md @@ -0,0 +1,352 @@ +--- +title: "Kafka Connect" +--- + + +# Kafka Connect + +[Kafka Connect](https://docs.confluent.io/platform/current/connect/index.html) is a popular framework for moving data +in and out of Kafka via connectors. There are many different connectors available, such as the S3 sink +for writing data from Kafka to S3 and Debezium source connectors for writing change data capture records from relational +databases to Kafka. + +It has a straightforward, decentralized, distributed architecture. A cluster consists of a number of worker processes, +and a connector runs tasks on these processes to perform the work. Connector deployment is configuration driven, so +generally no code needs to be written to run a connector. + +## Apache Iceberg Sink Connector + +The Apache Iceberg Sink Connector for Kafka Connect is a sink connector for writing data from Kafka into Iceberg tables. + +## Features + +* Commit coordination for centralized Iceberg commits +* Exactly-once delivery semantics +* Multi-table fan-out +* Automatic table creation and schema evolution +* Field name mapping via Iceberg’s column mapping functionality + +## Installation + +The connector zip archive is created as part of the Iceberg build. You can run the build via: +```bash +./gradlew -x test -x integrationTest clean build +``` +The zip archive will be found under `./kafka-connect/kafka-connect-runtime/build/distributions`. There is +one distribution that bundles the Hive Metastore client and related dependencies, and one that does not. +Copy the distribution archive into the Kafka Connect plugins directory on all nodes. + +## Requirements + +The sink relies on [KIP-447](https://cwiki.apache.org/confluence/display/KAFKA/KIP-447%3A+Producer+scalability+for+exactly+once+semantics) +for exactly-once semantics. This requires Kafka 2.5 or later. + +## Configuration + +| Property | Description | +|--------------------------------------------|------------------------------------------------------------------------------------------------------------------| +| iceberg.tables | Comma-separated list of destination tables | +| iceberg.tables.dynamic-enabled | Set to `true` to route to a table specified in `routeField` instead of using `routeRegex`, default is `false` | +| iceberg.tables.route-field | For multi-table fan-out, the name of the field used to route records to tables | +| iceberg.tables.default-commit-branch | Default branch for commits, main is used if not specified | +| iceberg.tables.default-id-columns | Default comma-separated list of columns that identify a row in tables (primary key) | +| iceberg.tables.default-partition-by | Default comma-separated list of partition field names to use when creating tables | +| iceberg.tables.auto-create-enabled | Set to `true` to automatically create destination tables, default is `false` | +| iceberg.tables.evolve-schema-enabled | Set to `true` to add any missing record fields to the table schema, default is `false` | +| iceberg.tables.schema-force-optional | Set to `true` to set columns as optional during table create and evolution, default is `false` to respect schema | +| iceberg.tables.schema-case-insensitive | Set to `true` to look up table columns by case-insensitive name, default is `false` for case-sensitive | +| iceberg.tables.auto-create-props.* | Properties set on new tables during auto-create | +| iceberg.tables.write-props.* | Properties passed through to Iceberg writer initialization, these take precedence | +| iceberg.table.\.commit-branch | Table-specific branch for commits, use `iceberg.tables.default-commit-branch` if not specified | +| iceberg.table.\
.id-columns | Comma-separated list of columns that identify a row in the table (primary key) | +| iceberg.table.\
.partition-by | Comma-separated list of partition fields to use when creating the table | +| iceberg.table.\
.route-regex | The regex used to match a record's `routeField` to a table | +| iceberg.control.topic | Name of the control topic, default is `control-iceberg` | +| iceberg.control.commit.interval-ms | Commit interval in msec, default is 300,000 (5 min) | +| iceberg.control.commit.timeout-ms | Commit timeout interval in msec, default is 30,000 (30 sec) | +| iceberg.control.commit.threads | Number of threads to use for commits, default is (cores * 2) | +| iceberg.catalog | Name of the catalog, default is `iceberg` | +| iceberg.catalog.* | Properties passed through to Iceberg catalog initialization | +| iceberg.hadoop-conf-dir | If specified, Hadoop config files in this directory will be loaded | +| iceberg.hadoop.* | Properties passed through to the Hadoop configuration | +| iceberg.kafka.* | Properties passed through to control topic Kafka client initialization | + +If `iceberg.tables.dynamic-enabled` is `false` (the default) then you must specify `iceberg.tables`. If +`iceberg.tables.dynamic-enabled` is `true` then you must specify `iceberg.tables.route-field` which will +contain the name of the table. + +### Kafka configuration + +By default the connector will attempt to use Kafka client config from the worker properties for connecting to +the control topic. If that config cannot be read for some reason, Kafka client settings +can be set explicitly using `iceberg.kafka.*` properties. + +#### Message format + +Messages should be converted to a struct or map using the appropriate Kafka Connect converter. + +### Catalog configuration + +The `iceberg.catalog.*` properties are required for connecting to the Iceberg catalog. The core catalog +types are included in the default distribution, including REST, Glue, DynamoDB, Hadoop, Nessie, +JDBC, and Hive. JDBC drivers are not included in the default distribution, so you will need to include +those if needed. When using a Hive catalog, you can use the distribution that includes the Hive metastore client, +otherwise you will need to include that yourself. + +To set the catalog type, you can set `iceberg.catalog.type` to `rest`, `hive`, or `hadoop`. For other +catalog types, you need to instead set `iceberg.catalog.catalog-impl` to the name of the catalog class. + +#### REST example + +``` +"iceberg.catalog.type": "rest", +"iceberg.catalog.uri": "https://catalog-service", +"iceberg.catalog.credential": "", +"iceberg.catalog.warehouse": "", +``` + +#### Hive example + +NOTE: Use the distribution that includes the HMS client (or include the HMS client yourself). Use `S3FileIO` when +using S3 for storage (the default is `HadoopFileIO` with `HiveCatalog`). +``` +"iceberg.catalog.type": "hive", +"iceberg.catalog.uri": "thrift://hive:9083", +"iceberg.catalog.io-impl": "org.apache.iceberg.aws.s3.S3FileIO", +"iceberg.catalog.warehouse": "s3a://bucket/warehouse", +"iceberg.catalog.client.region": "us-east-1", +"iceberg.catalog.s3.access-key-id": "", +"iceberg.catalog.s3.secret-access-key": "", +``` + +#### Glue example + +``` +"iceberg.catalog.catalog-impl": "org.apache.iceberg.aws.glue.GlueCatalog", +"iceberg.catalog.warehouse": "s3a://bucket/warehouse", +"iceberg.catalog.io-impl": "org.apache.iceberg.aws.s3.S3FileIO", +``` + +#### Nessie example + +``` +"iceberg.catalog.catalog-impl": "org.apache.iceberg.nessie.NessieCatalog", +"iceberg.catalog.uri": "http://localhost:19120/api/v2", +"iceberg.catalog.ref": "main", +"iceberg.catalog.warehouse": "s3a://bucket/warehouse", +"iceberg.catalog.io-impl": "org.apache.iceberg.aws.s3.S3FileIO", +``` + +#### Notes + +Depending on your setup, you may need to also set `iceberg.catalog.s3.endpoint`, `iceberg.catalog.s3.staging-dir`, +or `iceberg.catalog.s3.path-style-access`. See the [Iceberg docs](https://iceberg.apache.org/docs/latest/) for +full details on configuring catalogs. + +### Azure ADLS configuration example + +When using ADLS, Azure requires the passing of AZURE_CLIENT_ID, AZURE_TENANT_ID, and AZURE_CLIENT_SECRET for its Java SDK. +If you're running Kafka Connect in a container, be sure to inject those values as environment variables. See the +[Azure Identity Client library for Java](https://learn.microsoft.com/en-us/java/api/overview/azure/identity-readme?view=azure-java-stable) for more information. + +An example of these would be: +``` +AZURE_CLIENT_ID=e564f687-7b89-4b48-80b8-111111111111 +AZURE_TENANT_ID=95f2f365-f5b7-44b1-88a1-111111111111 +AZURE_CLIENT_SECRET="XXX" +``` +Where the CLIENT_ID is the Application ID of a registered application under +[App Registrations](https://portal.azure.com/#view/Microsoft_AAD_RegisteredApps/ApplicationsListBlade), the TENANT_ID is +from your [Azure Tenant Properties](https://portal.azure.com/#view/Microsoft_AAD_IAM/TenantProperties.ReactView), and +the CLIENT_SECRET is created within the "Certificates & Secrets" section, under "Manage" after choosing your specific +App Registration. You might have to choose "Client secrets" in the middle panel and the "+" in front of "New client secret" +to generate one. Be sure to set this variable to the Value and not the Id. + +It's also important that the App Registration is granted the Role Assignment "Storage Blob Data Contributor" in your +Storage Account's Access Control (IAM), or it won't be able to write new files there. + +Then, within the Connector's configuration, you'll want to include the following: + +``` +"iceberg.catalog.type": "rest", +"iceberg.catalog.uri": "https://catalog:8181", +"iceberg.catalog.warehouse": "abfss://storage-container-name@storageaccount.dfs.core.windows.net/warehouse", +"iceberg.catalog.io-impl": "org.apache.iceberg.azure.adlsv2.ADLSFileIO", +"iceberg.catalog.include-credentials": "true" +``` + +Where `storage-container-name` is the container name within your Azure Storage Account, `/warehouse` is the location +within that container where your Apache Iceberg files will be written by default (or if iceberg.tables.auto-create-enabled=true), +and the `include-credentials` parameter passes along the Azure Java client credentials along. This will configure the +Iceberg Sink connector to connect to the REST catalog implementation at `iceberg.catalog.uri` to obtain the required +Connection String for the ADLSv2 client + +### Google GCS configuration example + +By default, Application Default Credentials (ADC) will be used to connect to GCS. Details on how ADC works can +be found in the [Google Cloud documentation](https://cloud.google.com/docs/authentication/application-default-credentials). + +``` +"iceberg.catalog.type": "rest", +"iceberg.catalog.uri": "https://catalog:8181", +"iceberg.catalog.warehouse": "gs://bucket-name/warehouse", +"iceberg.catalog.io-impl": "org.apache.iceberg.google.gcs.GCSFileIO" +``` + +### Hadoop configuration + +When using HDFS or Hive, the sink will initialize the Hadoop configuration. First, config files +from the classpath are loaded. Next, if `iceberg.hadoop-conf-dir` is specified, config files +are loaded from that location. Finally, any `iceberg.hadoop.*` properties from the sink config are +applied. When merging these, the order of precedence is sink config > config dir > classpath. + +## Examples + +### Initial setup + +#### Source topic + +This assumes the source topic already exists and is named `events`. + +#### Control topic + +If your Kafka cluster has `auto.create.topics.enable` set to `true` (the default), then the control topic will be +automatically created. If not, then you will need to create the topic first. The default topic name is `control-iceberg`: +```bash +bin/kafka-topics \ + --command-config command-config.props \ + --bootstrap-server ${CONNECT_BOOTSTRAP_SERVERS} \ + --create \ + --topic control-iceberg \ + --partitions 1 +``` +*NOTE: Clusters running on Confluent Cloud have `auto.create.topics.enable` set to `false` by default.* + +#### Iceberg catalog configuration + +Configuration properties with the prefix `iceberg.catalog.` will be passed to Iceberg catalog initialization. +See the [Iceberg docs](https://iceberg.apache.org/docs/latest/) for details on how to configure +a particular catalog. + +### Single destination table + +This example writes all incoming records to a single table. + +#### Create the destination table + +```sql +CREATE TABLE default.events ( + id STRING, + type STRING, + ts TIMESTAMP, + payload STRING) +PARTITIONED BY (hours(ts)) +``` + +#### Connector config + +This example config connects to a Iceberg REST catalog. +```json +{ +"name": "events-sink", +"config": { + "connector.class": "org.apache.iceberg.connect.IcebergSinkConnector", + "tasks.max": "2", + "topics": "events", + "iceberg.tables": "default.events", + "iceberg.catalog.type": "rest", + "iceberg.catalog.uri": "https://localhost", + "iceberg.catalog.credential": "", + "iceberg.catalog.warehouse": "" + } +} +``` + +### Multi-table fan-out, static routing + +This example writes records with `type` set to `list` to the table `default.events_list`, and +writes records with `type` set to `create` to the table `default.events_create`. Other records +will be skipped. + +#### Create two destination tables + +```sql +CREATE TABLE default.events_list ( + id STRING, + type STRING, + ts TIMESTAMP, + payload STRING) +PARTITIONED BY (hours(ts)); + +CREATE TABLE default.events_create ( + id STRING, + type STRING, + ts TIMESTAMP, + payload STRING) +PARTITIONED BY (hours(ts)); +``` + +#### Connector config + +```json +{ +"name": "events-sink", +"config": { + "connector.class": "org.apache.iceberg.connect.IcebergSinkConnector", + "tasks.max": "2", + "topics": "events", + "iceberg.tables": "default.events_list,default.events_create", + "iceberg.tables.route-field": "type", + "iceberg.table.default.events_list.route-regex": "list", + "iceberg.table.default.events_create.route-regex": "create", + "iceberg.catalog.type": "rest", + "iceberg.catalog.uri": "https://localhost", + "iceberg.catalog.credential": "", + "iceberg.catalog.warehouse": "" + } +} +``` + +### Multi-table fan-out, dynamic routing + +This example writes to tables with names from the value in the `db_table` field. If a table with +the name does not exist, then the record will be skipped. For example, if the record's `db_table` +field is set to `default.events_list`, then the record is written to the `default.events_list` table. + +#### Create two destination tables + +See above for creating two tables. + +#### Connector config + +```json +{ +"name": "events-sink", +"config": { + "connector.class": "org.apache.iceberg.connect.IcebergSinkConnector", + "tasks.max": "2", + "topics": "events", + "iceberg.tables.dynamic-enabled": "true", + "iceberg.tables.route-field": "db_table", + "iceberg.catalog.type": "rest", + "iceberg.catalog.uri": "https://localhost", + "iceberg.catalog.credential": "", + "iceberg.catalog.warehouse": "" + } +} +``` diff --git a/docs/docs/partitioning.md b/docs/docs/partitioning.md index 09c20439d1b1..aa573b33e968 100644 --- a/docs/docs/partitioning.md +++ b/docs/docs/partitioning.md @@ -84,7 +84,7 @@ This leads to several problems: Iceberg produces partition values by taking a column value and optionally transforming it. Iceberg is responsible for converting `event_time` into `event_date`, and keeps track of the relationship. -Table partitioning is configured using these relationships. The `logs` table would be partitioned by `date(event_time)` and `level`. +Table partitioning is configured using these relationships. The `logs` table would be partitioned by `day(event_time)` and `level`. Because Iceberg doesn't require user-maintained partition columns, it can hide partitioning. Partition values are produced correctly every time and always used to speed up queries, when possible. Producers and consumers wouldn't even see `event_date`. diff --git a/docs/docs/spark-procedures.md b/docs/docs/spark-procedures.md index 31172fb531bd..0953e729a77b 100644 --- a/docs/docs/spark-procedures.md +++ b/docs/docs/spark-procedures.md @@ -312,6 +312,10 @@ Used to remove files which are not referenced in any metadata files of an Iceber | `location` | | string | Directory to look for files in (defaults to the table's location) | | `dry_run` | | boolean | When true, don't actually remove files (defaults to false) | | `max_concurrent_deletes` | | int | Size of the thread pool used for delete file actions (by default, no thread pool is used) | +| `file_list_view` | | string | Dataset to look for files in (skipping the directory listing) | +| `equal_schemes` | | map | Mapping of file system schemes to be considered equal. Key is a comma-separated list of schemes and value is a scheme (defaults to `map('s3a,s3n','s3')`). | +| `equal_authorities` | | map | Mapping of file system authorities to be considered equal. Key is a comma-separated list of authorities and value is an authority. | +| `prefix_mismatch_mode` | | string | Action behavior when location prefixes (schemes/authorities) mismatch:
  • ERROR - throw an exception. (default)
  • IGNORE - no action.
  • DELETE - delete files.
| #### Output @@ -331,6 +335,40 @@ Remove any files in the `tablelocation/data` folder which are not known to the t CALL catalog_name.system.remove_orphan_files(table => 'db.sample', location => 'tablelocation/data'); ``` +Remove any files in the `files_view` view which are not known to the table `db.sample`. +```java +Dataset compareToFileList = + spark + .createDataFrame(allFiles, FilePathLastModifiedRecord.class) + .withColumnRenamed("filePath", "file_path") + .withColumnRenamed("lastModified", "last_modified"); +String fileListViewName = "files_view"; +compareToFileList.createOrReplaceTempView(fileListViewName); +``` +```sql +CALL catalog_name.system.remove_orphan_files(table => 'db.sample', file_list_view => 'files_view'); +``` + +When a file matches references in metadata files except for location prefix (scheme/authority), an error is thrown by default. +The error can be ignored and the file will be skipped by setting `prefix_mismatch_mode` to `IGNORE`. +```sql +CALL catalog_name.system.remove_orphan_files(table => 'db.sample', prefix_mismatch_mode => 'IGNORE'); +``` + +The file can still be deleted by setting `prefix_mismatch_mode` to `DELETE`. +```sql +CALL catalog_name.system.remove_orphan_files(table => 'db.sample', prefix_mismatch_mode => 'DELETE'); +``` + +The file can also be deleted by considering the mismatched prefixes equal. +```sql +CALL catalog_name.system.remove_orphan_files(table => 'db.sample', equal_schemes => map('file', 'file1')); +``` + +```sql +CALL catalog_name.system.remove_orphan_files(table => 'db.sample', equal_authorities => map('ns1', 'ns2')); +``` + ### `rewrite_data_files` Iceberg tracks each data file in a table. More data files leads to more metadata stored in manifest files, and small data files causes an unnecessary amount of metadata and less efficient queries from file open costs. @@ -819,7 +857,7 @@ that provide additional information about the changes being tracked. These colum Here is an example of corresponding results. It shows that the first snapshot inserted 2 records, and the second snapshot deleted 1 record. -| id | name |_change_type | _change_ordinal | _change_snapshot_id | +| id | name |_change_type | _change_ordinal | _commit_snapshot_id | |---|--------|---|---|---| |1 | Alice |INSERT |0 |5390529835796506035| |2 | Bob |INSERT |0 |5390529835796506035| @@ -839,7 +877,7 @@ CALL spark_catalog.system.create_changelog_view( With the net changes, the above changelog view only contains the following row since Alice was inserted in the first snapshot and deleted in the second snapshot. -| id | name |_change_type | _change_ordinal | _change_snapshot_id | +| id | name |_change_type | _change_ordinal | _commit_snapshot_id | |---|--------|---|---|---| |2 | Bob |INSERT |0 |5390529835796506035| diff --git a/docs/mkdocs.yml b/docs/mkdocs.yml index 5e43aa1d26d9..c1c16dc50096 100644 --- a/docs/mkdocs.yml +++ b/docs/mkdocs.yml @@ -53,16 +53,20 @@ nav: - hive.md - Trino: https://trino.io/docs/current/connector/iceberg.html - Daft: daft.md - - Clickhouse: https://clickhouse.com/docs/en/engines/table-engines/integrations/iceberg + - ClickHouse: https://clickhouse.com/docs/en/engines/table-engines/integrations/iceberg - Presto: https://prestodb.io/docs/current/connector/iceberg.html - Dremio: https://docs.dremio.com/data-formats/apache-iceberg/ - Starrocks: https://docs.starrocks.io/en-us/latest/data_source/catalog/iceberg_catalog - Amazon Athena: https://docs.aws.amazon.com/athena/latest/ug/querying-iceberg.html - Amazon EMR: https://docs.aws.amazon.com/emr/latest/ReleaseGuide/emr-iceberg-use-cluster.html + - Amazon Data Firehose: https://docs.aws.amazon.com/firehose/latest/dev/apache-iceberg-destination.html + - Amazon Redshift: https://docs.aws.amazon.com/redshift/latest/dg/querying-iceberg.html - Google BigQuery: https://cloud.google.com/bigquery/docs/iceberg-tables - Snowflake: https://docs.snowflake.com/en/user-guide/tables-iceberg - Impala: https://impala.apache.org/docs/build/html/topics/impala_iceberg.html - Doris: https://doris.apache.org/docs/dev/lakehouse/datalake-analytics/iceberg + - Druid: https://druid.apache.org/docs/latest/development/extensions-contrib/iceberg/ + - Kafka Connect: kafka-connect.md - Integrations: - aws.md - dell.md diff --git a/flink/v1.18/build.gradle b/flink/v1.18/build.gradle index aac01c9c6931..83dc07523a3c 100644 --- a/flink/v1.18/build.gradle +++ b/flink/v1.18/build.gradle @@ -127,7 +127,7 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { } project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") { - apply plugin: 'io.github.goooler.shadow' + apply plugin: 'com.gradleup.shadow' tasks.jar.dependsOn tasks.shadowJar diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java index 1453753849ec..9b0c7a938920 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java @@ -70,6 +70,7 @@ public class FlinkCatalogFactory implements CatalogFactory { public static final String HADOOP_CONF_DIR = "hadoop-conf-dir"; public static final String DEFAULT_DATABASE = "default-database"; public static final String DEFAULT_DATABASE_NAME = "default"; + public static final String DEFAULT_CATALOG_NAME = "default_catalog"; public static final String BASE_NAMESPACE = "base-namespace"; public static final String TYPE = "type"; diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java index 7167859e600c..d5eea6706b39 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java @@ -53,6 +53,10 @@ public LongConfParser longConf() { return new LongConfParser(); } + public DoubleConfParser doubleConf() { + return new DoubleConfParser(); + } + public > EnumConfParser enumConfParser(Class enumClass) { return new EnumConfParser<>(enumClass); } @@ -135,6 +139,29 @@ public Long parseOptional() { } } + class DoubleConfParser extends ConfParser { + private Double defaultValue; + + @Override + protected DoubleConfParser self() { + return this; + } + + public DoubleConfParser defaultValue(double value) { + this.defaultValue = value; + return self(); + } + + public double parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(Double::parseDouble, defaultValue); + } + + public Double parseOptional() { + return parse(Double::parseDouble, null); + } + } + class StringConfParser extends ConfParser { private String defaultValue; diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java index 4790dc85bf28..afc09a719f77 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java @@ -58,7 +58,9 @@ public class FlinkSchemaUtil { private FlinkSchemaUtil() {} - /** @deprecated Use {@link #convert(ResolvedSchema)} instead. */ + /** + * @deprecated Use {@link #convert(ResolvedSchema)} instead. + */ @Deprecated public static Schema convert(TableSchema schema) { LogicalType schemaType = schema.toRowDataType().getLogicalType(); diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java index ca7b1120bc81..a31902d49a8b 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java @@ -26,6 +26,7 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.sink.shuffle.StatisticsType; /** * A class for common Iceberg configs for Flink writes. @@ -167,6 +168,26 @@ public DistributionMode distributionMode() { return DistributionMode.fromName(modeName); } + public StatisticsType rangeDistributionStatisticsType() { + String name = + confParser + .stringConf() + .option(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE.key()) + .flinkConfig(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE) + .defaultValue(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE.defaultValue()) + .parse(); + return StatisticsType.valueOf(name); + } + + public double rangeDistributionSortKeyBaseWeight() { + return confParser + .doubleConf() + .option(FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT.key()) + .flinkConfig(FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT) + .defaultValue(FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT.defaultValue()) + .parse(); + } + public int workerPoolSize() { return confParser .intConf() diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java index df73f2e09cac..c35286774874 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java @@ -23,6 +23,7 @@ import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.flink.sink.shuffle.StatisticsType; /** Flink sink write options */ public class FlinkWriteOptions { @@ -60,6 +61,19 @@ private FlinkWriteOptions() {} public static final ConfigOption DISTRIBUTION_MODE = ConfigOptions.key("distribution-mode").stringType().noDefaultValue(); + public static final ConfigOption RANGE_DISTRIBUTION_STATISTICS_TYPE = + ConfigOptions.key("range-distribution-statistics-type") + .stringType() + .defaultValue(StatisticsType.Auto.name()) + .withDescription("Type of statistics collection: Auto, Map, Sketch"); + + public static final ConfigOption RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT = + ConfigOptions.key("range-distribution-sort-key-base-weight") + .doubleType() + .defaultValue(0.0d) + .withDescription( + "Base weight for every sort key relative to target weight per writer task"); + // Branch to write to public static final ConfigOption BRANCH = ConfigOptions.key("branch").stringType().defaultValue(SnapshotRef.MAIN_BRANCH); diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java index 1019285018d0..34576a1e5c0b 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java @@ -237,10 +237,10 @@ public MapData getMap(int pos) { @Override public RowData getRow(int pos, int numFields) { - return isNullAt(pos) ? null : getStructRowData(pos, numFields); + return isNullAt(pos) ? null : getStructRowData(pos); } - private StructRowData getStructRowData(int pos, int numFields) { + private StructRowData getStructRowData(int pos) { return new StructRowData( type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class)); } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java index 769af7d77140..5cd43a46de37 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java @@ -53,15 +53,20 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Partitioning; import org.apache.iceberg.Schema; import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.SortOrder; import org.apache.iceberg.Table; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.FlinkWriteConf; import org.apache.iceberg.flink.FlinkWriteOptions; import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.sink.shuffle.DataStatisticsOperatorFactory; +import org.apache.iceberg.flink.sink.shuffle.RangePartitioner; +import org.apache.iceberg.flink.sink.shuffle.StatisticsOrRecord; +import org.apache.iceberg.flink.sink.shuffle.StatisticsType; import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; -import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -233,15 +238,68 @@ public Builder flinkConf(ReadableConfig config) { * @return {@link Builder} to connect the iceberg table. */ public Builder distributionMode(DistributionMode mode) { - Preconditions.checkArgument( - !DistributionMode.RANGE.equals(mode), - "Flink does not support 'range' write distribution mode now."); if (mode != null) { writeOptions.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), mode.modeName()); } return this; } + /** + * Range distribution needs to collect statistics about data distribution to properly shuffle + * the records in relatively balanced way. In general, low cardinality should use {@link + * StatisticsType#Map} and high cardinality should use {@link StatisticsType#Sketch} Refer to + * {@link StatisticsType} Javadoc for more details. + * + *

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

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

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

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

Default is {@code 0.0%}. + */ + public Builder rangeDistributionSortKeyBaseWeight(double weight) { + writeOptions.put( + FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT.key(), Double.toString(weight)); + return this; + } + /** * Configuring the write parallel number for iceberg stream writer. * @@ -349,18 +407,20 @@ private DataStreamSink chainIcebergOperators() { // Find out the equality field id list based on the user-provided equality field column names. List equalityFieldIds = checkAndGetEqualityFieldIds(); - // Convert the requested flink table schema to flink row type. RowType flinkRowType = toFlinkRowType(table.schema(), tableSchema); + int writerParallelism = + flinkWriteConf.writeParallelism() == null + ? rowDataInput.getParallelism() + : flinkWriteConf.writeParallelism(); // Distribute the records from input data stream based on the write.distribution-mode and // equality fields. DataStream distributeStream = - distributeDataStream( - rowDataInput, equalityFieldIds, table.spec(), table.schema(), flinkRowType); + distributeDataStream(rowDataInput, equalityFieldIds, flinkRowType, writerParallelism); // Add parallel writers that append rows to files - SingleOutputStreamOperator writerStream = - appendWriter(distributeStream, flinkRowType, equalityFieldIds); + SingleOutputStreamOperator writerStream = + appendWriter(distributeStream, flinkRowType, equalityFieldIds, writerParallelism); // Add single-parallelism committer that commits files // after successful checkpoint or end of input @@ -426,7 +486,7 @@ private DataStreamSink appendDummySink( } private SingleOutputStreamOperator appendCommitter( - SingleOutputStreamOperator writerStream) { + SingleOutputStreamOperator writerStream) { IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter( tableLoader, @@ -446,8 +506,11 @@ private SingleOutputStreamOperator appendCommitter( return committerStream; } - private SingleOutputStreamOperator appendWriter( - DataStream input, RowType flinkRowType, List equalityFieldIds) { + private SingleOutputStreamOperator appendWriter( + DataStream input, + RowType flinkRowType, + List equalityFieldIds, + int writerParallelism) { // Validate the equality fields and partition fields if we enable the upsert mode. if (flinkWriteConf.upsertMode()) { Preconditions.checkState( @@ -481,17 +544,13 @@ private SingleOutputStreamOperator appendWriter( IcebergStreamWriter streamWriter = createStreamWriter(tableSupplier, flinkWriteConf, flinkRowType, equalityFieldIds); - int parallelism = - flinkWriteConf.writeParallelism() == null - ? input.getParallelism() - : flinkWriteConf.writeParallelism(); - SingleOutputStreamOperator writerStream = + SingleOutputStreamOperator writerStream = input .transform( operatorName(ICEBERG_STREAM_WRITER_NAME), - TypeInformation.of(WriteResult.class), + TypeInformation.of(FlinkWriteResult.class), streamWriter) - .setParallelism(parallelism); + .setParallelism(writerParallelism); if (uidPrefix != null) { writerStream = writerStream.uid(uidPrefix + "-writer"); } @@ -501,12 +560,15 @@ private SingleOutputStreamOperator appendWriter( private DataStream distributeDataStream( DataStream input, List equalityFieldIds, - PartitionSpec partitionSpec, - Schema iSchema, - RowType flinkRowType) { + RowType flinkRowType, + int writerParallelism) { DistributionMode writeMode = flinkWriteConf.distributionMode(); - LOG.info("Write distribution mode is '{}'", writeMode.modeName()); + + Schema iSchema = table.schema(); + PartitionSpec partitionSpec = table.spec(); + SortOrder sortOrder = table.sortOrder(); + switch (writeMode) { case NONE: if (equalityFieldIds.isEmpty()) { @@ -548,21 +610,52 @@ private DataStream distributeDataStream( } case RANGE: - if (equalityFieldIds.isEmpty()) { + // Ideally, exception should be thrown in the combination of range distribution and + // equality fields. Primary key case should use hash distribution mode. + // Keep the current behavior of falling back to keyBy for backward compatibility. + if (!equalityFieldIds.isEmpty()) { LOG.warn( - "Fallback to use 'none' distribution mode, because there are no equality fields set " - + "and {}=range is not supported yet in flink", - WRITE_DISTRIBUTION_MODE); - return input; - } else { - LOG.info( - "Distribute rows by equality fields, because there are equality fields set " - + "and{}=range is not supported yet in flink", + "Hash distribute rows by equality fields, even though {}=range is set. " + + "Range distribution for primary keys are not always safe in " + + "Flink streaming writer.", WRITE_DISTRIBUTION_MODE); return input.keyBy( new EqualityFieldKeySelector(iSchema, flinkRowType, equalityFieldIds)); } + // range distribute by partition key or sort key if table has an SortOrder + Preconditions.checkState( + sortOrder.isSorted() || partitionSpec.isPartitioned(), + "Invalid write distribution mode: range. Need to define sort order or partition spec."); + if (sortOrder.isUnsorted()) { + sortOrder = Partitioning.sortOrderFor(partitionSpec); + LOG.info("Construct sort order from partition spec"); + } + + LOG.info("Range distribute rows by sort order: {}", sortOrder); + StatisticsType statisticsType = flinkWriteConf.rangeDistributionStatisticsType(); + SingleOutputStreamOperator shuffleStream = + input + .transform( + operatorName("range-shuffle"), + TypeInformation.of(StatisticsOrRecord.class), + new DataStatisticsOperatorFactory( + iSchema, + sortOrder, + writerParallelism, + statisticsType, + flinkWriteConf.rangeDistributionSortKeyBaseWeight())) + // Set the parallelism same as input operator to encourage chaining + .setParallelism(input.getParallelism()); + if (uidPrefix != null) { + shuffleStream = shuffleStream.uid(uidPrefix + "-shuffle"); + } + + return shuffleStream + .partitionCustom(new RangePartitioner(iSchema, sortOrder), r -> r) + .filter(StatisticsOrRecord::hasRecord) + .map(StatisticsOrRecord::record); + default: throw new RuntimeException("Unrecognized " + WRITE_DISTRIBUTION_MODE + ": " + writeMode); } @@ -577,12 +670,9 @@ static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) { TypeUtil.validateWriteSchema(schema, writeSchema, true, true); // We use this flink schema to read values from RowData. The flink's TINYINT and SMALLINT will - // be promoted to - // iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT (backend by 1 - // 'byte'), we will - // read 4 bytes rather than 1 byte, it will mess up the byte array in BinaryRowData. So here - // we must use flink - // schema. + // be promoted to iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT + // (backend by 1 'byte'), we will read 4 bytes rather than 1 byte, it will mess up the + // byte array in BinaryRowData. So here we must use flink schema. return (RowType) requestedSchema.toRowDataType().getLogicalType(); } else { return FlinkSchemaUtil.convert(schema); diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.java new file mode 100644 index 000000000000..317fb169ae1b --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.Serializable; +import org.apache.iceberg.io.WriteResult; + +public class FlinkWriteResult implements Serializable { + private final long checkpointId; + private final WriteResult writeResult; + + public FlinkWriteResult(long checkpointId, WriteResult writeResult) { + this.checkpointId = checkpointId; + this.writeResult = writeResult; + } + + public long checkpointId() { + return checkpointId; + } + + public WriteResult writeResult() { + return writeResult; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java index b9bceaa9311d..7108c2008341 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java @@ -63,7 +63,7 @@ import org.slf4j.LoggerFactory; class IcebergFilesCommitter extends AbstractStreamOperator - implements OneInputStreamOperator, BoundedOneInput { + implements OneInputStreamOperator, BoundedOneInput { private static final long serialVersionUID = 1L; private static final long INITIAL_CHECKPOINT_ID = -1L; @@ -96,7 +96,7 @@ class IcebergFilesCommitter extends AbstractStreamOperator // The completed files cache for current checkpoint. Once the snapshot barrier received, it will // be flushed to the 'dataFilesPerCheckpoint'. - private final List writeResultsOfCurrentCkpt = Lists.newArrayList(); + private final Map> writeResultsSinceLastSnapshot = Maps.newHashMap(); private final String branch; // It will have an unique identifier for one job. @@ -212,7 +212,8 @@ public void snapshotState(StateSnapshotContext context) throws Exception { // Update the checkpoint state. long startNano = System.nanoTime(); - dataFilesPerCheckpoint.put(checkpointId, writeToManifest(checkpointId)); + writeToManifestUptoLatestCheckpoint(checkpointId); + // Reset the snapshot state to the latest state. checkpointsState.clear(); checkpointsState.add(dataFilesPerCheckpoint); @@ -220,8 +221,6 @@ public void snapshotState(StateSnapshotContext context) throws Exception { jobIdState.clear(); jobIdState.add(flinkJobId); - // Clear the local buffer for current checkpoint. - writeResultsOfCurrentCkpt.clear(); committerMetrics.checkpointDuration( TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); } @@ -426,30 +425,45 @@ private void commitOperation( } @Override - public void processElement(StreamRecord element) { - this.writeResultsOfCurrentCkpt.add(element.getValue()); + public void processElement(StreamRecord element) { + FlinkWriteResult flinkWriteResult = element.getValue(); + List writeResults = + writeResultsSinceLastSnapshot.computeIfAbsent( + flinkWriteResult.checkpointId(), k -> Lists.newArrayList()); + writeResults.add(flinkWriteResult.writeResult()); } @Override public void endInput() throws IOException { // Flush the buffered data files into 'dataFilesPerCheckpoint' firstly. - long currentCheckpointId = Long.MAX_VALUE; - dataFilesPerCheckpoint.put(currentCheckpointId, writeToManifest(currentCheckpointId)); - writeResultsOfCurrentCkpt.clear(); - + long currentCheckpointId = IcebergStreamWriter.END_INPUT_CHECKPOINT_ID; + writeToManifestUptoLatestCheckpoint(currentCheckpointId); commitUpToCheckpoint(dataFilesPerCheckpoint, flinkJobId, operatorUniqueId, currentCheckpointId); } + private void writeToManifestUptoLatestCheckpoint(long checkpointId) throws IOException { + if (!writeResultsSinceLastSnapshot.containsKey(checkpointId)) { + dataFilesPerCheckpoint.put(checkpointId, EMPTY_MANIFEST_DATA); + } + + for (Map.Entry> writeResultsOfCheckpoint : + writeResultsSinceLastSnapshot.entrySet()) { + dataFilesPerCheckpoint.put( + writeResultsOfCheckpoint.getKey(), + writeToManifest(writeResultsOfCheckpoint.getKey(), writeResultsOfCheckpoint.getValue())); + } + + // Clear the local buffer for current checkpoint. + writeResultsSinceLastSnapshot.clear(); + } + /** * Write all the complete data files to a newly created manifest file and return the manifest's * avro serialized bytes. */ - private byte[] writeToManifest(long checkpointId) throws IOException { - if (writeResultsOfCurrentCkpt.isEmpty()) { - return EMPTY_MANIFEST_DATA; - } - - WriteResult result = WriteResult.builder().addAll(writeResultsOfCurrentCkpt).build(); + private byte[] writeToManifest(long checkpointId, List writeResults) + throws IOException { + WriteResult result = WriteResult.builder().addAll(writeResults).build(); DeltaManifests deltaManifests = FlinkManifestUtil.writeCompletedFiles( result, () -> manifestOutputFileFactory.create(checkpointId), spec); diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java index 9ea0349fb057..bb5efe982ee1 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java @@ -29,10 +29,11 @@ import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -class IcebergStreamWriter extends AbstractStreamOperator - implements OneInputStreamOperator, BoundedOneInput { +class IcebergStreamWriter extends AbstractStreamOperator + implements OneInputStreamOperator, BoundedOneInput { private static final long serialVersionUID = 1L; + static final long END_INPUT_CHECKPOINT_ID = Long.MAX_VALUE; private final String fullTableName; private final TaskWriterFactory taskWriterFactory; @@ -63,7 +64,7 @@ public void open() { @Override public void prepareSnapshotPreBarrier(long checkpointId) throws Exception { - flush(); + flush(checkpointId); this.writer = taskWriterFactory.create(); } @@ -89,7 +90,7 @@ public void endInput() throws IOException { // Note that if the task is not closed after calling endInput, checkpoint may be triggered again // causing files to be sent repeatedly, the writer is marked as null after the last file is sent // to guard against duplicated writes. - flush(); + flush(END_INPUT_CHECKPOINT_ID); } @Override @@ -102,7 +103,7 @@ public String toString() { } /** close all open files and emit files to downstream committer operator */ - private void flush() throws IOException { + private void flush(long checkpointId) throws IOException { if (writer == null) { return; } @@ -110,7 +111,7 @@ private void flush() throws IOException { long startNano = System.nanoTime(); WriteResult result = writer.complete(); writerMetrics.updateFlushResult(result); - output.collect(new StreamRecord<>(result)); + output.collect(new StreamRecord<>(new FlinkWriteResult(checkpointId, result))); writerMetrics.flushDuration(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); // Set writer to null to prevent duplicate flushes in the corner case of diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java index da5e6e7627ae..47a74a2018fb 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink; +import java.util.Locale; import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Supplier; @@ -59,6 +60,7 @@ class ManifestOutputFileFactory { private String generatePath(long checkpointId) { return FileFormat.AVRO.addExtension( String.format( + Locale.ROOT, "%s-%s-%05d-%d-%d-%05d", flinkJobId, operatorUniqueId, diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java index 5525f02c873e..95c2328f032a 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java @@ -186,7 +186,9 @@ private boolean isComplete() { return subtaskSet.size() == parallelism; } - /** @return false if duplicate */ + /** + * @return false if duplicate + */ private boolean merge(int subtask, DataStatistics taskStatistics) { if (subtaskSet.contains(subtask)) { return false; diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java index 4bfde7204acf..e2a282efd82e 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.sink.shuffle; import java.util.Comparator; +import java.util.Locale; import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; @@ -265,8 +266,10 @@ private void sendGlobalStatisticsToSubtasks(GlobalStatistics statistics) { } }, String.format( + Locale.ROOT, "Failed to send operator %s coordinator global data statistics for checkpoint %d", - operatorName, statistics.checkpointId())); + operatorName, + statistics.checkpointId())); } @SuppressWarnings("FutureReturnValueIgnored") @@ -290,8 +293,11 @@ private void handleRequestGlobalStatisticsEvent(int subtask, RequestGlobalStatis } }, String.format( + Locale.ROOT, "Failed to send operator %s coordinator global data statistics to requesting subtask %d for checkpoint %d", - operatorName, subtask, globalStatistics.checkpointId())); + operatorName, + subtask, + globalStatistics.checkpointId())); } else { LOG.info( "Ignore global statistics request from subtask {} as statistics not available", subtask); @@ -318,8 +324,11 @@ public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEven } }, String.format( + Locale.ROOT, "handling operator event %s from subtask %d (#%d)", - event.getClass(), subtask, attemptNumber)); + event.getClass(), + subtask, + attemptNumber)); } @Override @@ -339,7 +348,7 @@ public void checkpointCoordinator(long checkpointId, CompletableFuture r completedStatistics, completedStatisticsSerializer)); } }, - String.format("taking checkpoint %d", checkpointId)); + String.format(Locale.ROOT, "taking checkpoint %d", checkpointId)); } @Override @@ -381,7 +390,8 @@ public void subtaskReset(int subtask, long checkpointId) { this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); subtaskGateways.reset(subtask); }, - String.format("handling subtask %d recovery to checkpoint %d", subtask, checkpointId)); + String.format( + Locale.ROOT, "handling subtask %d recovery to checkpoint %d", subtask, checkpointId)); } @Override @@ -397,7 +407,7 @@ public void executionAttemptFailed(int subtask, int attemptNumber, @Nullable Thr this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); subtaskGateways.unregisterSubtaskGateway(subtask, attemptNumber); }, - String.format("handling subtask %d (#%d) failure", subtask, attemptNumber)); + String.format(Locale.ROOT, "handling subtask %d (#%d) failure", subtask, attemptNumber)); } @Override @@ -411,7 +421,10 @@ public void executionAttemptReady(int subtask, int attemptNumber, SubtaskGateway subtaskGateways.registerSubtaskGateway(gateway); }, String.format( - "making event gateway to subtask %d (#%d) available", subtask, attemptNumber)); + Locale.ROOT, + "making event gateway to subtask %d (#%d) available", + subtask, + attemptNumber)); } @VisibleForTesting diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java new file mode 100644 index 000000000000..dc147bf36d13 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; + +@Internal +public class DataStatisticsOperatorFactory extends AbstractStreamOperatorFactory + implements CoordinatedOperatorFactory, + OneInputStreamOperatorFactory { + + private final Schema schema; + private final SortOrder sortOrder; + private final int downstreamParallelism; + private final StatisticsType type; + private final double closeFileCostWeightPercentage; + + public DataStatisticsOperatorFactory( + Schema schema, + SortOrder sortOrder, + int downstreamParallelism, + StatisticsType type, + double closeFileCostWeightPercentage) { + this.schema = schema; + this.sortOrder = sortOrder; + this.downstreamParallelism = downstreamParallelism; + this.type = type; + this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; + } + + @Override + public OperatorCoordinator.Provider getCoordinatorProvider( + String operatorName, OperatorID operatorID) { + return new DataStatisticsCoordinatorProvider( + operatorName, + operatorID, + schema, + sortOrder, + downstreamParallelism, + type, + closeFileCostWeightPercentage); + } + + @SuppressWarnings("unchecked") + @Override + public > T createStreamOperator( + StreamOperatorParameters parameters) { + OperatorID operatorId = parameters.getStreamConfig().getOperatorID(); + String operatorName = parameters.getStreamConfig().getOperatorName(); + OperatorEventGateway gateway = + parameters.getOperatorEventDispatcher().getOperatorEventGateway(operatorId); + + DataStatisticsOperator rangeStatisticsOperator = + new DataStatisticsOperator( + operatorName, schema, sortOrder, gateway, downstreamParallelism, type); + + rangeStatisticsOperator.setup( + parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); + parameters + .getOperatorEventDispatcher() + .registerEventHandler(operatorId, rangeStatisticsOperator); + + return (T) rangeStatisticsOperator; + } + + @SuppressWarnings("rawtypes") + @Override + public Class getStreamOperatorClass(ClassLoader classLoader) { + return DataStatisticsOperator.class; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java index ce17e1964392..ce94bec14860 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java @@ -27,7 +27,9 @@ class RequestGlobalStatisticsEvent implements OperatorEvent { this.signature = null; } - /** @param signature hashCode of the subtask's existing global statistics */ + /** + * @param signature hashCode of the subtask's existing global statistics + */ RequestGlobalStatisticsEvent(int signature) { this.signature = signature; } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java index d1d75019fa2e..61c6973463ef 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java @@ -23,6 +23,7 @@ import java.math.BigInteger; import java.nio.ByteBuffer; import java.util.List; +import java.util.Locale; import java.util.Objects; import java.util.Set; import java.util.UUID; @@ -170,7 +171,8 @@ public void serialize(SortKey record, DataOutputView target) throws IOException default: // SortKey transformation is a flattened struct without list and map throw new UnsupportedOperationException( - String.format("Field %d has unsupported field type: %s", fieldId, typeId)); + String.format( + Locale.ROOT, "Field %d has unsupported field type: %s", fieldId, typeId)); } } } @@ -239,7 +241,8 @@ public SortKey deserialize(SortKey reuse, DataInputView source) throws IOExcepti default: // SortKey transformation is a flattened struct without list and map throw new UnsupportedOperationException( - String.format("Field %d has unsupported field type: %s", fieldId, typeId)); + String.format( + Locale.ROOT, "Field %d has unsupported field type: %s", fieldId, typeId)); } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java index 91d975349b19..3beda960cec8 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.util.Iterator; +import java.util.Locale; import org.apache.flink.annotation.Internal; import org.apache.iceberg.CombinedScanTask; import org.apache.iceberg.FileScanTask; @@ -95,8 +96,11 @@ public void seek(int startingFileOffset, long startingRecordOffset) { } else { throw new IllegalStateException( String.format( + Locale.ROOT, "Invalid starting record offset %d for file %d from CombinedScanTask: %s", - startingRecordOffset, startingFileOffset, combinedTask)); + startingRecordOffset, + startingFileOffset, + combinedTask)); } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java index b1431a32dd20..b5a1ba85a6cb 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java @@ -28,6 +28,7 @@ import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.iceberg.Schema; @@ -46,6 +47,14 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.PropertyUtil; +/** + * Flink source builder for old {@link SourceFunction} implementation. + * + * @deprecated since 1.7.0, will be removed in 2.0.0. Use {@link IcebergSource} instead, which + * implement the newer FLIP-27 source interface. This class implements the old {@link + * SourceFunction} that has been marked as deprecated in Flink since Aug 2023. + */ +@Deprecated public class FlinkSource { private FlinkSource() {} @@ -124,7 +133,9 @@ public Builder setAll(Map properties) { return this; } - /** @deprecated Use {@link #setAll} instead. */ + /** + * @deprecated Use {@link #setAll} instead. + */ @Deprecated public Builder properties(Map properties) { readOptions.putAll(properties); diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index ccbd0d9997ed..6461d8207122 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -27,7 +27,8 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; -import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.Source; import org.apache.flink.api.connector.source.SourceReader; @@ -37,6 +38,9 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.flink.util.Preconditions; @@ -61,10 +65,12 @@ import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer; import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator; import org.apache.iceberg.flink.source.reader.ColumnStatsWatermarkExtractor; +import org.apache.iceberg.flink.source.reader.ConverterReaderFunction; import org.apache.iceberg.flink.source.reader.IcebergSourceReader; import org.apache.iceberg.flink.source.reader.IcebergSourceReaderMetrics; import org.apache.iceberg.flink.source.reader.MetaDataReaderFunction; import org.apache.iceberg.flink.source.reader.ReaderFunction; +import org.apache.iceberg.flink.source.reader.RowDataConverter; import org.apache.iceberg.flink.source.reader.RowDataReaderFunction; import org.apache.iceberg.flink.source.reader.SerializableRecordEmitter; import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; @@ -72,13 +78,13 @@ import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.flink.source.split.SplitComparators; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.ThreadPools; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -@Experimental public class IcebergSource implements Source { private static final Logger LOG = LoggerFactory.getLogger(IcebergSource.class); @@ -95,6 +101,11 @@ public class IcebergSource implements Source emitter; private final String tableName; + // cache the discovered splits by planSplitsForBatch, which can be called twice. And they come + // from two different threads: (1) source/stream construction by main thread (2) enumerator + // creation. Hence need volatile here. + private volatile List batchSplits; + IcebergSource( TableLoader tableLoader, ScanContext scanContext, @@ -130,16 +141,26 @@ private String planningThreadName() { return tableName + "-" + UUID.randomUUID(); } + /** + * Cache the enumerated splits for batch execution to avoid double planning as there are two code + * paths obtaining splits: (1) infer parallelism (2) enumerator creation. + */ private List planSplitsForBatch(String threadName) { + if (batchSplits != null) { + return batchSplits; + } + ExecutorService workerPool = ThreadPools.newWorkerPool(threadName, scanContext.planParallelism()); try (TableLoader loader = tableLoader.clone()) { loader.open(); - List splits = + this.batchSplits = FlinkSplitPlanner.planIcebergSourceSplits(loader.loadTable(), scanContext, workerPool); LOG.info( - "Discovered {} splits from table {} during job initialization", splits.size(), tableName); - return splits; + "Discovered {} splits from table {} during job initialization", + batchSplits.size(), + tableName); + return batchSplits; } catch (IOException e) { throw new UncheckedIOException("Failed to close table loader", e); } finally { @@ -205,26 +226,69 @@ private SplitEnumerator createEnumer // Only do scan planning if nothing is restored from checkpoint state List splits = planSplitsForBatch(planningThreadName()); assigner.onDiscoveredSplits(splits); + // clear the cached splits after enumerator creation as they won't be needed anymore + this.batchSplits = null; } return new StaticIcebergEnumerator(enumContext, assigner); } } + private boolean shouldInferParallelism() { + return !scanContext.isStreaming(); + } + + private int inferParallelism(ReadableConfig flinkConf, StreamExecutionEnvironment env) { + int parallelism = + SourceUtil.inferParallelism( + flinkConf, + scanContext.limit(), + () -> { + List splits = planSplitsForBatch(planningThreadName()); + return splits.size(); + }); + + if (env.getMaxParallelism() > 0) { + parallelism = Math.min(parallelism, env.getMaxParallelism()); + } + + return parallelism; + } + + /** + * Create a source builder. + * + * @deprecated since 1.7.0. Will be removed in 2.0.0; use{@link IcebergSource#forRowData()} or + * {@link IcebergSource#forOutputType(RowDataConverter)} instead + */ + @Deprecated public static Builder builder() { return new Builder<>(); } + /** Create a source builder for RowData output type. */ public static Builder forRowData() { return new Builder<>(); } + /** + * Create a source builder that would convert {@link RowData} to the output type {@code T}. + * + * @param converter convert {@link RowData} to output type {@code T} + * @param output type + * @return an IcebergSource builder + */ + public static Builder forOutputType(RowDataConverter converter) { + return new Builder().converter(converter); + } + public static class Builder { private TableLoader tableLoader; private Table table; private SplitAssignerFactory splitAssignerFactory; private SerializableComparator splitComparator; private ReaderFunction readerFunction; + private RowDataConverter converter; private ReadableConfig flinkConfig = new Configuration(); private final ScanContext.Builder contextBuilder = ScanContext.builder(); private TableSchema projectedFlinkSchema; @@ -255,11 +319,28 @@ public Builder splitComparator( return this; } + /** + * @deprecated since 1.7.0. Will be removed in 2.0.0; use{@link + * IcebergSource#forOutputType(RowDataConverter)} instead to produce output type other than + * {@link RowData}. + */ + @Deprecated public Builder readerFunction(ReaderFunction newReaderFunction) { + Preconditions.checkState( + converter == null, + "Cannot set reader function when builder was created via IcebergSource.forOutputType(Converter)"); this.readerFunction = newReaderFunction; return this; } + /** + * Don't need to be public. It is set by {@link IcebergSource#forOutputType(RowDataConverter)}. + */ + private Builder converter(RowDataConverter newConverter) { + this.converter = newConverter; + return this; + } + public Builder flinkConfig(ReadableConfig config) { this.flinkConfig = config; return this; @@ -464,7 +545,9 @@ public Builder watermarkColumnTimeUnit(TimeUnit timeUnit) { return this; } - /** @deprecated Use {@link #setAll} instead. */ + /** + * @deprecated Use {@link #setAll} instead. + */ @Deprecated public Builder properties(Map properties) { readOptions.putAll(properties); @@ -482,6 +565,10 @@ public IcebergSource build() { } contextBuilder.resolveConfig(table, readOptions, flinkConfig); + contextBuilder.exposeLocality( + SourceUtil.isLocalityEnabled(table, flinkConfig, exposeLocality)); + contextBuilder.planParallelism( + flinkConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE)); Schema icebergSchema = table.schema(); if (projectedFlinkSchema != null) { contextBuilder.project(FlinkSchemaUtil.convert(icebergSchema, projectedFlinkSchema)); @@ -506,25 +593,7 @@ public IcebergSource build() { ScanContext context = contextBuilder.build(); context.validate(); if (readerFunction == null) { - if (table instanceof BaseMetadataTable) { - MetaDataReaderFunction rowDataReaderFunction = - new MetaDataReaderFunction( - flinkConfig, table.schema(), context.project(), table.io(), table.encryption()); - this.readerFunction = (ReaderFunction) rowDataReaderFunction; - } else { - RowDataReaderFunction rowDataReaderFunction = - new RowDataReaderFunction( - flinkConfig, - table.schema(), - context.project(), - context.nameMapping(), - context.caseSensitive(), - table.io(), - table.encryption(), - context.filters(), - context.limit()); - this.readerFunction = (ReaderFunction) rowDataReaderFunction; - } + this.readerFunction = readerFunction(context); } if (splitAssignerFactory == null) { @@ -545,5 +614,75 @@ public IcebergSource build() { table, emitter); } + + /** + * Build the {@link IcebergSource} and create a {@link DataStream} from the source. Watermark + * strategy is set to {@link WatermarkStrategy#noWatermarks()}. + * + * @return data stream from the Iceberg source + */ + public DataStream buildStream(StreamExecutionEnvironment env) { + // buildStream should only be called with RowData or Converter paths. + Preconditions.checkState( + readerFunction == null, + "Cannot set reader function when building a data stream from the source"); + IcebergSource source = build(); + TypeInformation outputTypeInfo = + outputTypeInfo(converter, table.schema(), source.scanContext.project()); + DataStreamSource stream = + env.fromSource(source, WatermarkStrategy.noWatermarks(), source.name(), outputTypeInfo); + if (source.shouldInferParallelism()) { + stream = stream.setParallelism(source.inferParallelism(flinkConfig, env)); + } + + return stream; + } + + private static TypeInformation outputTypeInfo( + RowDataConverter converter, Schema tableSchema, Schema projected) { + if (converter != null) { + return converter.getProducedType(); + } else { + // output type is RowData + Schema readSchema = projected != null ? projected : tableSchema; + return (TypeInformation) + FlinkCompatibilityUtil.toTypeInfo(FlinkSchemaUtil.convert(readSchema)); + } + } + + private ReaderFunction readerFunction(ScanContext context) { + if (table instanceof BaseMetadataTable) { + MetaDataReaderFunction rowDataReaderFunction = + new MetaDataReaderFunction( + flinkConfig, table.schema(), context.project(), table.io(), table.encryption()); + return (ReaderFunction) rowDataReaderFunction; + } else { + if (converter == null) { + return (ReaderFunction) + new RowDataReaderFunction( + flinkConfig, + table.schema(), + context.project(), + context.nameMapping(), + context.caseSensitive(), + table.io(), + table.encryption(), + context.filters(), + context.limit()); + } else { + return new ConverterReaderFunction<>( + converter, + flinkConfig, + table.schema(), + context.project(), + context.nameMapping(), + context.caseSensitive(), + table.io(), + table.encryption(), + context.filters(), + context.limit()); + } + } + } } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java index 610657e8d47b..65adce77d9f9 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java @@ -23,11 +23,8 @@ import java.util.Map; import java.util.Optional; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.connector.ChangelogMode; @@ -128,26 +125,18 @@ private DataStream createDataStream(StreamExecutionEnvironment execEnv) .build(); } - private DataStreamSource createFLIP27Stream(StreamExecutionEnvironment env) { + private DataStream createFLIP27Stream(StreamExecutionEnvironment env) { SplitAssignerType assignerType = readableConfig.get(FlinkConfigOptions.TABLE_EXEC_SPLIT_ASSIGNER_TYPE); - IcebergSource source = - IcebergSource.forRowData() - .tableLoader(loader) - .assignerFactory(assignerType.factory()) - .properties(properties) - .project(getProjectedSchema()) - .limit(limit) - .filters(filters) - .flinkConfig(readableConfig) - .build(); - DataStreamSource stream = - env.fromSource( - source, - WatermarkStrategy.noWatermarks(), - source.name(), - TypeInformation.of(RowData.class)); - return stream; + return IcebergSource.forRowData() + .tableLoader(loader) + .assignerFactory(assignerType.factory()) + .properties(properties) + .project(getProjectedSchema()) + .limit(limit) + .filters(filters) + .flinkConfig(readableConfig) + .buildStream(env); } private TableSchema getProjectedSchema() { diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java index c958604c004a..391633924264 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java @@ -104,10 +104,7 @@ public static class RewriteMap extends RichMapFunction taskWriterFactory; private final RowDataFileScanTaskReader rowDataReader; @@ -119,10 +116,7 @@ public RewriteMap( boolean caseSensitive, EncryptionManager encryptionManager, TaskWriterFactory taskWriterFactory) { - this.schema = schema; - this.nameMapping = nameMapping; this.io = io; - this.caseSensitive = caseSensitive; this.encryptionManager = encryptionManager; this.taskWriterFactory = taskWriterFactory; this.rowDataReader = diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java index 280a126a46ce..fc310606dee9 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java @@ -22,6 +22,7 @@ import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicReference; @@ -76,6 +77,7 @@ public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname // Iceberg source uses custom split request event to piggyback finished split ids. throw new UnsupportedOperationException( String.format( + Locale.ROOT, "Received invalid default split request event " + "from subtask %d as Iceberg source uses custom split request event", subtaskId)); @@ -92,8 +94,10 @@ public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { } else { throw new IllegalArgumentException( String.format( + Locale.ROOT, "Received unknown event from subtask %d: %s", - subtaskId, sourceEvent.getClass().getCanonicalName())); + subtaskId, + sourceEvent.getClass().getCanonicalName())); } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java index c7021b9c6847..c50c3854ee14 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java @@ -37,6 +37,7 @@ public class ContinuousIcebergEnumerator extends AbstractIcebergEnumerator { private static final Logger LOG = LoggerFactory.getLogger(ContinuousIcebergEnumerator.class); + /** * This is hardcoded, as {@link ScanContext#maxPlanningSnapshotCount()} could be the knob to * control the total number of snapshots worth of splits tracked by assigner. diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.java new file mode 100644 index 000000000000..b158b0871a53 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.avro.RowDataToAvroConverters; +import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; +import org.apache.flink.formats.avro.typeutils.GenericRecordAvroTypeInfo; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.flink.FlinkSchemaUtil; + +public class AvroGenericRecordConverter implements RowDataConverter { + private final Schema avroSchema; + private final RowDataToAvroConverters.RowDataToAvroConverter flinkConverter; + private final TypeInformation outputTypeInfo; + + private AvroGenericRecordConverter(Schema avroSchema, RowType rowType) { + this.avroSchema = avroSchema; + this.flinkConverter = RowDataToAvroConverters.createConverter(rowType); + this.outputTypeInfo = new GenericRecordAvroTypeInfo(avroSchema); + } + + public static AvroGenericRecordConverter fromIcebergSchema( + org.apache.iceberg.Schema icebergSchema, String tableName) { + RowType rowType = FlinkSchemaUtil.convert(icebergSchema); + Schema avroSchema = AvroSchemaUtil.convert(icebergSchema, tableName); + return new AvroGenericRecordConverter(avroSchema, rowType); + } + + public static AvroGenericRecordConverter fromAvroSchema(Schema avroSchema, String tableName) { + DataType dataType = AvroSchemaConverter.convertToDataType(avroSchema.toString()); + LogicalType logicalType = TypeConversions.fromDataToLogicalType(dataType); + RowType rowType = RowType.of(logicalType.getChildren().toArray(new LogicalType[0])); + return new AvroGenericRecordConverter(avroSchema, rowType); + } + + @Override + public GenericRecord apply(RowData rowData) { + return (GenericRecord) flinkConverter.convert(avroSchema, rowData); + } + + @Override + public TypeInformation getProducedType() { + return outputTypeInfo; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java index 66e59633fff2..f89e5ce13474 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java @@ -28,13 +28,21 @@ import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.flink.source.AvroGenericRecordFileScanTaskReader; import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.IcebergSource; import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; import org.apache.iceberg.flink.source.RowDataToAvroGenericRecordConverter; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -/** Read Iceberg rows as {@link GenericRecord}. */ +/** + * Read Iceberg rows as {@link GenericRecord}. + * + * @deprecated since 1.7.0. Will be removed in 2.0.0; use {@link + * IcebergSource#forOutputType(RowDataConverter)} and {@link AvroGenericRecordConverter} + * instead. + */ +@Deprecated public class AvroGenericRecordReaderFunction extends DataIteratorReaderFunction { private final String tableName; private final Schema readSchema; diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java new file mode 100644 index 000000000000..e1e7c17d63c5 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import java.util.List; +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Schema; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.InputFilesDecryptor; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.FileScanTaskReader; +import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +@Internal +public class ConverterReaderFunction extends DataIteratorReaderFunction { + private final RowDataConverter converter; + private final Schema tableSchema; + private final Schema readSchema; + private final String nameMapping; + private final boolean caseSensitive; + private final FileIO io; + private final EncryptionManager encryption; + private final List filters; + private final long limit; + + private transient RecordLimiter recordLimiter = null; + + public ConverterReaderFunction( + RowDataConverter converter, + ReadableConfig config, + Schema tableSchema, + Schema projectedSchema, + String nameMapping, + boolean caseSensitive, + FileIO io, + EncryptionManager encryption, + List filters, + long limit) { + super(new ListDataIteratorBatcher<>(config)); + this.converter = converter; + this.tableSchema = tableSchema; + this.readSchema = readSchema(tableSchema, projectedSchema); + this.nameMapping = nameMapping; + this.caseSensitive = caseSensitive; + this.io = io; + this.encryption = encryption; + this.filters = filters; + this.limit = limit; + } + + @Override + protected DataIterator createDataIterator(IcebergSourceSplit split) { + RowDataFileScanTaskReader rowDataReader = + new RowDataFileScanTaskReader(tableSchema, readSchema, nameMapping, caseSensitive, filters); + return new LimitableDataIterator<>( + new ConverterFileScanTaskReader<>(rowDataReader, converter), + split.task(), + io, + encryption, + lazyLimiter()); + } + + private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { + Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); + return projectedSchema == null ? tableSchema : projectedSchema; + } + + /** Lazily create RecordLimiter to avoid the need to make it serializable */ + private RecordLimiter lazyLimiter() { + if (recordLimiter == null) { + this.recordLimiter = RecordLimiter.create(limit); + } + + return recordLimiter; + } + + private static class ConverterFileScanTaskReader implements FileScanTaskReader { + private final RowDataFileScanTaskReader rowDataReader; + private final RowDataConverter converter; + + ConverterFileScanTaskReader( + RowDataFileScanTaskReader rowDataReader, RowDataConverter converter) { + this.rowDataReader = rowDataReader; + this.converter = converter; + } + + @Override + public CloseableIterator open( + FileScanTask fileScanTask, InputFilesDecryptor inputFilesDecryptor) { + return CloseableIterator.transform( + rowDataReader.open(fileScanTask, inputFilesDecryptor), converter); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java index 9c20494fdbcd..bcd72e25036b 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java @@ -85,7 +85,7 @@ public RecordsWithSplitIds> fetch() throws IOException { } else { // return an empty result, which will lead to split fetch to be idle. // SplitFetcherManager will then close idle fetcher. - return new RecordsBySplits(Collections.emptyMap(), Collections.emptySet()); + return new RecordsBySplits<>(Collections.emptyMap(), Collections.emptySet()); } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java index 6ac92592b6aa..10e7d2037a30 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.source.reader; +import java.util.Locale; import org.apache.flink.annotation.Internal; /** @@ -73,6 +74,6 @@ public void record(T nextRecord) { @Override public String toString() { - return String.format("%s @ %d + %d", record, fileOffset, recordOffset); + return String.format(Locale.ROOT, "%s @ %d + %d", record, fileOffset, recordOffset); } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java new file mode 100644 index 000000000000..98bb7e981840 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import java.io.Serializable; +import java.util.function.Function; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.table.data.RowData; + +/** + * Convert RowData to a different output type. + * + * @param output type + */ +public interface RowDataConverter + extends Function, ResultTypeQueryable, Serializable {} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java index d4b0f9e1977d..d90d1dc88c91 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.source.split; import java.io.IOException; +import java.util.Locale; import org.apache.flink.annotation.Internal; import org.apache.flink.core.io.SimpleVersionedSerializer; @@ -54,6 +55,7 @@ public IcebergSourceSplit deserialize(int version, byte[] serialized) throws IOE default: throw new IOException( String.format( + Locale.ROOT, "Failed to deserialize IcebergSourceSplit. " + "Encountered unsupported version: %d. Supported version are [1]", version)); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java index 91ed3c4adea3..062ff68d5d85 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java @@ -118,26 +118,4 @@ protected String getFullQualifiedTableName(String tableName) { static String getURI(HiveConf conf) { return conf.get(HiveConf.ConfVars.METASTOREURIS.varname); } - - static String toWithClause(Map props) { - StringBuilder builder = new StringBuilder(); - builder.append("("); - int propCount = 0; - for (Map.Entry entry : props.entrySet()) { - if (propCount > 0) { - builder.append(","); - } - builder - .append("'") - .append(entry.getKey()) - .append("'") - .append("=") - .append("'") - .append(entry.getValue()) - .append("'"); - propCount++; - } - builder.append(")"); - return builder.toString(); - } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SqlBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SqlBase.java new file mode 100644 index 000000000000..9411ea4f7d71 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SqlBase.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink; + +import static org.apache.iceberg.flink.FlinkCatalogFactory.DEFAULT_CATALOG_NAME; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import java.util.Map; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +public abstract class SqlBase { + protected abstract TableEnvironment getTableEnv(); + + protected static TableResult exec(TableEnvironment env, String query, Object... args) { + return env.executeSql(String.format(query, args)); + } + + protected TableResult exec(String query, Object... args) { + return exec(getTableEnv(), query, args); + } + + protected List sql(String query, Object... args) { + TableResult tableResult = exec(query, args); + try (CloseableIterator iter = tableResult.collect()) { + return Lists.newArrayList(iter); + } catch (Exception e) { + throw new RuntimeException("Failed to collect table result", e); + } + } + + protected void assertSameElements(Iterable expected, Iterable actual) { + assertThat(actual).isNotNull().containsExactlyInAnyOrderElementsOf(expected); + } + + protected void assertSameElements(String message, Iterable expected, Iterable actual) { + assertThat(actual).isNotNull().as(message).containsExactlyInAnyOrderElementsOf(expected); + } + + /** + * We can not drop currently used catalog after FLINK-29677, so we have make sure that we do not + * use the current catalog before dropping it. This method switches to the 'default_catalog' and + * drops the one requested. + * + * @param catalogName The catalog to drop + * @param ifExists If we should use the 'IF EXISTS' when dropping the catalog + */ + protected void dropCatalog(String catalogName, boolean ifExists) { + sql("USE CATALOG %s", DEFAULT_CATALOG_NAME); + sql("DROP CATALOG %s %s", ifExists ? "IF EXISTS" : "", catalogName); + } + + /** + * We can not drop currently used database after FLINK-33226, so we have make sure that we do not + * use the current database before dropping it. This method switches to the default database in + * the default catalog, and then it and drops the one requested. + * + * @param database The database to drop + * @param ifExists If we should use the 'IF EXISTS' when dropping the database + */ + protected void dropDatabase(String database, boolean ifExists) { + String currentCatalog = getTableEnv().getCurrentCatalog(); + sql("USE CATALOG %s", DEFAULT_CATALOG_NAME); + sql("USE %s", getTableEnv().listDatabases()[0]); + sql("USE CATALOG %s", currentCatalog); + sql("DROP DATABASE %s %s", ifExists ? "IF EXISTS" : "", database); + } + + protected static String toWithClause(Map props) { + StringBuilder builder = new StringBuilder(); + builder.append("("); + int propCount = 0; + for (Map.Entry entry : props.entrySet()) { + if (propCount > 0) { + builder.append(","); + } + builder + .append("'") + .append(entry.getKey()) + .append("'") + .append("=") + .append("'") + .append(entry.getValue()) + .append("'"); + propCount++; + } + builder.append(")"); + return builder.toString(); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java index a74226092f38..401960c3591b 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink; +import static org.apache.iceberg.flink.FlinkCatalogFactory.DEFAULT_CATALOG_NAME; import static org.assertj.core.api.Assertions.assertThat; import java.nio.file.Path; @@ -26,7 +27,6 @@ import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.TableResult; import org.apache.flink.test.junit5.MiniClusterExtension; -import org.apache.flink.test.util.TestBaseUtils; import org.apache.flink.types.Row; import org.apache.flink.util.CloseableIterator; import org.apache.hadoop.hive.conf.HiveConf; @@ -40,7 +40,7 @@ import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.api.io.TempDir; -public abstract class TestBase extends TestBaseUtils { +public abstract class TestBase extends SqlBase { @RegisterExtension public static MiniClusterExtension miniClusterExtension = @@ -71,6 +71,7 @@ public static void stopMetastore() throws Exception { TestBase.catalog = null; } + @Override protected TableEnvironment getTableEnv() { if (tEnv == null) { synchronized (this) { @@ -122,7 +123,23 @@ protected void assertSameElements(String message, Iterable expected, Iterab * @param ifExists If we should use the 'IF EXISTS' when dropping the catalog */ protected void dropCatalog(String catalogName, boolean ifExists) { - sql("USE CATALOG default_catalog"); + sql("USE CATALOG %s", DEFAULT_CATALOG_NAME); sql("DROP CATALOG %s %s", ifExists ? "IF EXISTS" : "", catalogName); } + + /** + * We can not drop currently used database after FLINK-33226, so we have make sure that we do not + * use the current database before dropping it. This method switches to the default database in + * the default catalog, and then it and drops the one requested. + * + * @param database The database to drop + * @param ifExists If we should use the 'IF EXISTS' when dropping the database + */ + protected void dropDatabase(String database, boolean ifExists) { + String currentCatalog = getTableEnv().getCurrentCatalog(); + sql("USE CATALOG %s", DEFAULT_CATALOG_NAME); + sql("USE %s", getTableEnv().listDatabases()[0]); + sql("USE CATALOG %s", currentCatalog); + sql("DROP DATABASE %s %s", ifExists ? "IF EXISTS" : "", database); + } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java index b73300e3f170..2978a92945a2 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java @@ -18,36 +18,21 @@ */ package org.apache.iceberg.flink; -import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assumptions.assumeThat; -import java.util.Collections; import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import org.apache.flink.api.dag.Transformation; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.Expressions; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.table.api.internal.TableEnvironmentImpl; -import org.apache.flink.table.operations.ModifyOperation; -import org.apache.flink.table.planner.delegation.PlannerBase; -import org.apache.flink.types.Row; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DistributionMode; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Parameter; import org.apache.iceberg.Parameters; import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.flink.source.BoundedTableFactory; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -122,7 +107,7 @@ public void before() { @AfterEach public void clean() { sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + dropDatabase(flinkDatabase, true); BoundedTableFactory.clearDataSets(); super.clean(); } @@ -169,39 +154,6 @@ public void testOverwriteTable() throws Exception { icebergTable, Lists.newArrayList(SimpleDataUtil.createRecord(2, "b"))); } - @TestTemplate - public void testWriteParallelism() throws Exception { - List dataSet = - IntStream.range(1, 1000) - .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) - .flatMap(List::stream) - .collect(Collectors.toList()); - String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); - sql( - "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" - + " WITH ('connector'='BoundedSource', 'data-id'='%s')", - SOURCE_TABLE, dataId); - - PlannerBase planner = (PlannerBase) ((TableEnvironmentImpl) getTableEnv()).getPlanner(); - String insertSQL = - String.format( - "INSERT INTO %s /*+ OPTIONS('write-parallelism'='1') */ SELECT * FROM %s", - TABLE_NAME, SOURCE_TABLE); - ModifyOperation operation = (ModifyOperation) planner.getParser().parse(insertSQL).get(0); - Transformation dummySink = planner.translate(Collections.singletonList(operation)).get(0); - Transformation committer = dummySink.getInputs().get(0); - Transformation writer = committer.getInputs().get(0); - - assertThat(writer.getParallelism()).as("Should have the expected 1 parallelism.").isEqualTo(1); - writer - .getInputs() - .forEach( - input -> - assertThat(input.getParallelism()) - .as("Should have the expected parallelism.") - .isEqualTo(isStreamingJob ? 2 : 4)); - } - @TestTemplate public void testReplacePartitions() throws Exception { assumeThat(isStreamingJob) @@ -289,70 +241,4 @@ public void testInsertIntoPartition() throws Exception { sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); } } - - @TestTemplate - public void testHashDistributeMode() throws Exception { - String tableName = "test_hash_distribution_mode"; - Map tableProps = - ImmutableMap.of( - "write.format.default", - format.name(), - TableProperties.WRITE_DISTRIBUTION_MODE, - DistributionMode.HASH.modeName()); - - // Initialize a BoundedSource table to precisely emit those rows in only one checkpoint. - List dataSet = - IntStream.range(1, 1000) - .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) - .flatMap(List::stream) - .collect(Collectors.toList()); - String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); - sql( - "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" - + " WITH ('connector'='BoundedSource', 'data-id'='%s')", - SOURCE_TABLE, dataId); - - assertThat(sql("SELECT * FROM %s", SOURCE_TABLE)) - .as("Should have the expected rows in source table.") - .containsExactlyInAnyOrderElementsOf(dataSet); - - sql( - "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH %s", - tableName, toWithClause(tableProps)); - - try { - // Insert data set. - sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); - - assertThat(sql("SELECT * FROM %s", tableName)) - .as("Should have the expected rows in sink table.") - .containsExactlyInAnyOrderElementsOf(dataSet); - - // Sometimes we will have more than one checkpoint if we pass the auto checkpoint interval, - // thus producing multiple snapshots. Here we assert that each snapshot has only 1 file per - // partition. - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, tableName)); - Map> snapshotToDataFiles = SimpleDataUtil.snapshotToDataFiles(table); - for (List dataFiles : snapshotToDataFiles.values()) { - if (dataFiles.isEmpty()) { - continue; - } - - assertThat( - SimpleDataUtil.matchingPartitions( - dataFiles, table.spec(), ImmutableMap.of("data", "aaa"))) - .hasSize(1); - assertThat( - SimpleDataUtil.matchingPartitions( - dataFiles, table.spec(), ImmutableMap.of("data", "bbb"))) - .hasSize(1); - assertThat( - SimpleDataUtil.matchingPartitions( - dataFiles, table.spec(), ImmutableMap.of("data", "ccc"))) - .hasSize(1); - } - } finally { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - } - } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java new file mode 100644 index 000000000000..b63547d433a4 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java @@ -0,0 +1,336 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink; + +import static org.apache.iceberg.flink.FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HADOOP; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.io.File; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.internal.TableEnvironmentImpl; +import org.apache.flink.table.operations.ModifyOperation; +import org.apache.flink.table.planner.delegation.PlannerBase; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.source.BoundedTableFactory; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +/** + * This class tests the more extended features of Flink sink. Extract them separately since it is + * unnecessary to test all the parameters combinations in {@link TestFlinkTableSink}, like catalog + * types, namespaces, file format, streaming/batch. Those combinations explode exponentially. Each + * test method in {@link TestFlinkTableSink} runs 21 combinations, which are expensive and slow. + */ +@ExtendWith(ParameterizedTestExtension.class) +public class TestFlinkTableSinkExtended extends SqlBase { + protected static final String CATALOG = "testhadoop"; + protected static final String DATABASE = "db"; + protected static final String TABLE = "tbl"; + + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + private static final String SOURCE_TABLE = "default_catalog.default_database.bounded_source"; + private static final String FLINK_DATABASE = CATALOG + "." + DATABASE; + private static final Namespace ICEBERG_NAMESPACE = Namespace.of(new String[] {DATABASE}); + + @TempDir protected File warehouseRoot; + + protected HadoopCatalog catalog = null; + + private TableEnvironment tEnv; + + @Parameter protected boolean isStreamingJob; + + @Parameters(name = "isStreamingJob={0}") + protected static List parameters() { + return Arrays.asList(new Boolean[] {true}, new Boolean[] {false}); + } + + protected synchronized TableEnvironment getTableEnv() { + if (tEnv == null) { + EnvironmentSettings.Builder settingsBuilder = EnvironmentSettings.newInstance(); + if (isStreamingJob) { + settingsBuilder.inStreamingMode(); + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); + env.enableCheckpointing(400); + env.setMaxParallelism(2); + env.setParallelism(2); + tEnv = StreamTableEnvironment.create(env, settingsBuilder.build()); + } else { + settingsBuilder.inBatchMode(); + tEnv = TableEnvironment.create(settingsBuilder.build()); + } + } + return tEnv; + } + + @BeforeEach + public void before() { + String warehouseLocation = "file:" + warehouseRoot.getPath(); + this.catalog = new HadoopCatalog(new Configuration(), warehouseLocation); + Map config = Maps.newHashMap(); + config.put("type", "iceberg"); + config.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, ICEBERG_CATALOG_TYPE_HADOOP); + config.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation); + sql("CREATE CATALOG %s WITH %s", CATALOG, toWithClause(config)); + + sql("CREATE DATABASE %s", FLINK_DATABASE); + sql("USE CATALOG %s", CATALOG); + sql("USE %s", DATABASE); + sql( + "CREATE TABLE %s (id int, data varchar) with ('write.format.default'='%s')", + TABLE, FileFormat.PARQUET.name()); + } + + @AfterEach + public void clean() throws Exception { + sql("DROP TABLE IF EXISTS %s.%s", FLINK_DATABASE, TABLE); + dropDatabase(FLINK_DATABASE, true); + BoundedTableFactory.clearDataSets(); + + dropCatalog(CATALOG, true); + catalog.close(); + } + + @TestTemplate + public void testWriteParallelism() { + List dataSet = + IntStream.range(1, 1000) + .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) + .flatMap(List::stream) + .collect(Collectors.toList()); + String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); + sql( + "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" + + " WITH ('connector'='BoundedSource', 'data-id'='%s')", + SOURCE_TABLE, dataId); + + PlannerBase planner = (PlannerBase) ((TableEnvironmentImpl) getTableEnv()).getPlanner(); + String insertSQL = + String.format( + "INSERT INTO %s /*+ OPTIONS('write-parallelism'='1') */ SELECT * FROM %s", + TABLE, SOURCE_TABLE); + ModifyOperation operation = (ModifyOperation) planner.getParser().parse(insertSQL).get(0); + Transformation dummySink = planner.translate(Collections.singletonList(operation)).get(0); + Transformation committer = dummySink.getInputs().get(0); + Transformation writer = committer.getInputs().get(0); + + assertThat(writer.getParallelism()).as("Should have the expected 1 parallelism.").isEqualTo(1); + writer + .getInputs() + .forEach( + input -> + assertThat(input.getParallelism()) + .as("Should have the expected parallelism.") + .isEqualTo(isStreamingJob ? 2 : 4)); + } + + @TestTemplate + public void testHashDistributeMode() throws Exception { + // Initialize a BoundedSource table to precisely emit those rows in only one checkpoint. + List dataSet = + IntStream.range(1, 1000) + .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) + .flatMap(List::stream) + .collect(Collectors.toList()); + String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); + sql( + "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" + + " WITH ('connector'='BoundedSource', 'data-id'='%s')", + SOURCE_TABLE, dataId); + + assertThat(sql("SELECT * FROM %s", SOURCE_TABLE)) + .as("Should have the expected rows in source table.") + .containsExactlyInAnyOrderElementsOf(dataSet); + + Map tableProps = + ImmutableMap.of( + "write.format.default", + FileFormat.PARQUET.name(), + TableProperties.WRITE_DISTRIBUTION_MODE, + DistributionMode.HASH.modeName()); + + String tableName = "test_hash_distribution_mode"; + sql( + "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH %s", + tableName, toWithClause(tableProps)); + + try { + // Insert data set. + sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); + + assertThat(sql("SELECT * FROM %s", tableName)) + .as("Should have the expected rows in sink table.") + .containsExactlyInAnyOrderElementsOf(dataSet); + + // Sometimes we will have more than one checkpoint if we pass the auto checkpoint interval, + // thus producing multiple snapshots. Here we assert that each snapshot has only 1 file per + // partition. + Table table = catalog.loadTable(TableIdentifier.of(ICEBERG_NAMESPACE, tableName)); + Map> snapshotToDataFiles = SimpleDataUtil.snapshotToDataFiles(table); + for (List dataFiles : snapshotToDataFiles.values()) { + if (dataFiles.isEmpty()) { + continue; + } + + assertThat( + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "aaa"))) + .hasSize(1); + assertThat( + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "bbb"))) + .hasSize(1); + assertThat( + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "ccc"))) + .hasSize(1); + } + } finally { + sql("DROP TABLE IF EXISTS %s.%s", FLINK_DATABASE, tableName); + } + } + + @TestTemplate + public void testRangeDistributionPartitionColumn() { + // Range partitioner currently only works with streaming writes (with checkpoints) + assumeThat(isStreamingJob).isTrue(); + + // Initialize a BoundedSource table to precisely emit those rows in only one checkpoint. + List> rowsPerCheckpoint = + IntStream.range(1, 6) + .mapToObj( + checkpointId -> { + List charRows = Lists.newArrayList(); + // emit 26x10 rows for each checkpoint cycle + for (int i = 0; i < 10; ++i) { + for (char c = 'a'; c <= 'z'; c++) { + charRows.add(Row.of(c - 'a', String.valueOf(c))); + } + } + return charRows; + }) + .collect(Collectors.toList()); + List flattenedRows = + rowsPerCheckpoint.stream().flatMap(List::stream).collect(Collectors.toList()); + + String dataId = BoundedTableFactory.registerDataSet(rowsPerCheckpoint); + sql( + "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" + + " WITH ('connector'='BoundedSource', 'data-id'='%s')", + SOURCE_TABLE, dataId); + + assertThat(sql("SELECT * FROM %s", SOURCE_TABLE)) + .as("Should have the expected rows in source table.") + .containsExactlyInAnyOrderElementsOf(flattenedRows); + + Map tableProps = + ImmutableMap.of( + "write.format.default", + FileFormat.PARQUET.name(), + TableProperties.WRITE_DISTRIBUTION_MODE, + DistributionMode.RANGE.modeName()); + + String tableName = "test_hash_distribution_mode"; + sql( + "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH %s", + tableName, toWithClause(tableProps)); + + try { + // Insert data set. + sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); + + assertThat(sql("SELECT * FROM %s", tableName)) + .as("Should have the expected rows in sink table.") + .containsExactlyInAnyOrderElementsOf(flattenedRows); + + Table table = catalog.loadTable(TableIdentifier.of(ICEBERG_NAMESPACE, tableName)); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(5); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + for (Snapshot snapshot : rangePartitionedCycles) { + List addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // range partition results in each partition only assigned to one writer task + // maybe less than 26 partitions as BoundedSource doesn't always precisely + // control the checkpoint boundary. + // It is hard to precisely control the test condition in SQL tests. + // Here only minimal safe assertions are applied to avoid flakiness. + // If there are no shuffling, the number of data files could be as high as + // 26 * 4 as the default parallelism is set to 4 for the mini cluster. + assertThat(addedDataFiles).hasSizeLessThanOrEqualTo(26); + } + } finally { + sql("DROP TABLE IF EXISTS %s.%s", FLINK_DATABASE, tableName); + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java index b709c0058f7d..47f5485df879 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java @@ -248,20 +248,12 @@ public void testCreateTableUnderDefaultDatabase() { public void testCatalogDatabaseConflictWithFlinkDatabase() { sql("CREATE DATABASE IF NOT EXISTS `%s`", databaseName()); sql("USE `%s`", databaseName()); - - try { - testCreateConnectorTable(); - // Ensure that the table was created under the specific database. - assertThatThrownBy( - () -> sql("CREATE TABLE `default_catalog`.`%s`.`%s`", databaseName(), TABLE_NAME)) - .isInstanceOf(org.apache.flink.table.api.TableException.class) - .hasMessageStartingWith("Could not execute CreateTable in path"); - } finally { - sql("DROP TABLE IF EXISTS `%s`.`%s`", databaseName(), TABLE_NAME); - if (!isDefaultDatabaseName()) { - sql("DROP DATABASE `%s`", databaseName()); - } - } + testCreateConnectorTable(); + // Ensure that the table was created under the specific database. + assertThatThrownBy( + () -> sql("CREATE TABLE `default_catalog`.`%s`.`%s`", databaseName(), TABLE_NAME)) + .isInstanceOf(org.apache.flink.table.api.TableException.class) + .hasMessageStartingWith("Could not execute CreateTable in path"); } @TestTemplate @@ -327,10 +319,6 @@ private String databaseName() { return properties.getOrDefault("catalog-database", "default_database"); } - private String toWithClause(Map props) { - return CatalogTestBase.toWithClause(props); - } - private String createWarehouse() { try { return String.format( diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java index 8faae1b05a4e..3299e7a97776 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java @@ -40,7 +40,6 @@ import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.io.BaseTaskWriter; import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -210,8 +209,10 @@ public void testCompressionOrc() throws Exception { .containsEntry(TableProperties.ORC_COMPRESSION_STRATEGY, "speed"); } - private static OneInputStreamOperatorTestHarness createIcebergStreamWriter( - Table icebergTable, TableSchema flinkSchema, Map override) throws Exception { + private static OneInputStreamOperatorTestHarness + createIcebergStreamWriter( + Table icebergTable, TableSchema flinkSchema, Map override) + throws Exception { RowType flinkRowType = FlinkSink.toFlinkRowType(icebergTable.schema(), flinkSchema); FlinkWriteConf flinkWriteConfig = new FlinkWriteConf( @@ -219,7 +220,7 @@ private static OneInputStreamOperatorTestHarness createIce IcebergStreamWriter streamWriter = FlinkSink.createStreamWriter(() -> icebergTable, flinkWriteConfig, flinkRowType, null); - OneInputStreamOperatorTestHarness harness = + OneInputStreamOperatorTestHarness harness = new OneInputStreamOperatorTestHarness<>(streamWriter, 1, 1, 0); harness.setup(); @@ -230,7 +231,7 @@ private static OneInputStreamOperatorTestHarness createIce private static Map appenderProperties( Table table, TableSchema schema, Map override) throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter(table, schema, override)) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java index 61ab087f2ca3..b778037c559c 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java @@ -18,20 +18,11 @@ */ package org.apache.iceberg.flink.sink; -import static org.apache.iceberg.flink.TestFixtures.DATABASE; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - import java.io.IOException; -import java.util.Collections; import java.util.List; -import java.util.Map; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; -import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; import org.apache.iceberg.DistributionMode; import org.apache.iceberg.FileFormat; @@ -39,37 +30,19 @@ import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.flink.FlinkWriteOptions; -import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.jupiter.api.extension.RegisterExtension; @ExtendWith(ParameterizedTestExtension.class) public class TestFlinkIcebergSink extends TestFlinkIcebergSinkBase { - - @RegisterExtension - public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = - MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); - - @RegisterExtension - private static final HadoopCatalogExtension CATALOG_EXTENSION = - new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); - - private TableLoader tableLoader; - @Parameter(index = 0) private FileFormat format; @@ -99,7 +72,7 @@ public static Object[][] parameters() { @BeforeEach public void before() throws IOException { - table = + this.table = CATALOG_EXTENSION .catalog() .createTable( @@ -110,14 +83,14 @@ public void before() throws IOException { : PartitionSpec.unpartitioned(), ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); - env = + this.env = StreamExecutionEnvironment.getExecutionEnvironment( MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100) .setParallelism(parallelism) .setMaxParallelism(parallelism); - tableLoader = CATALOG_EXTENSION.tableLoader(); + this.tableLoader = CATALOG_EXTENSION.tableLoader(); } @TestTemplate @@ -140,246 +113,13 @@ public void testWriteRowData() throws Exception { SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); } - private void testWriteRow(TableSchema tableSchema, DistributionMode distributionMode) - throws Exception { - List rows = createRows(""); - DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); - - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .table(table) - .tableLoader(tableLoader) - .tableSchema(tableSchema) - .writeParallelism(parallelism) - .distributionMode(distributionMode) - .append(); - - // Execute the program. - env.execute("Test Iceberg DataStream."); - - SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); - } - - private int partitionFiles(String partition) throws IOException { - return SimpleDataUtil.partitionDataFiles(table, ImmutableMap.of("data", partition)).size(); - } - @TestTemplate public void testWriteRow() throws Exception { - testWriteRow(null, DistributionMode.NONE); + testWriteRow(parallelism, null, DistributionMode.NONE); } @TestTemplate public void testWriteRowWithTableSchema() throws Exception { - testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); - } - - @TestTemplate - public void testJobNoneDistributeMode() throws Exception { - table - .updateProperties() - .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) - .commit(); - - testWriteRow(null, DistributionMode.NONE); - - if (parallelism > 1) { - if (partitioned) { - int files = partitionFiles("aaa") + partitionFiles("bbb") + partitionFiles("ccc"); - assertThat(files).isGreaterThan(3); - } - } - } - - @TestTemplate - public void testJobHashDistributionMode() { - table - .updateProperties() - .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) - .commit(); - - assertThatThrownBy(() -> testWriteRow(null, DistributionMode.RANGE)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Flink does not support 'range' write distribution mode now."); - } - - @TestTemplate - public void testJobNullDistributionMode() throws Exception { - table - .updateProperties() - .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) - .commit(); - - testWriteRow(null, null); - - if (partitioned) { - assertThat(partitionFiles("aaa")).isEqualTo(1); - assertThat(partitionFiles("bbb")).isEqualTo(1); - assertThat(partitionFiles("ccc")).isEqualTo(1); - } - } - - @TestTemplate - public void testPartitionWriteMode() throws Exception { - testWriteRow(null, DistributionMode.HASH); - if (partitioned) { - assertThat(partitionFiles("aaa")).isEqualTo(1); - assertThat(partitionFiles("bbb")).isEqualTo(1); - assertThat(partitionFiles("ccc")).isEqualTo(1); - } - } - - @TestTemplate - public void testShuffleByPartitionWithSchema() throws Exception { - testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.HASH); - if (partitioned) { - assertThat(partitionFiles("aaa")).isEqualTo(1); - assertThat(partitionFiles("bbb")).isEqualTo(1); - assertThat(partitionFiles("ccc")).isEqualTo(1); - } - } - - @TestTemplate - public void testTwoSinksInDisjointedDAG() throws Exception { - Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); - - Table leftTable = - CATALOG_EXTENSION - .catalog() - .createTable( - TableIdentifier.of("left"), - SimpleDataUtil.SCHEMA, - partitioned - ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() - : PartitionSpec.unpartitioned(), - props); - TableLoader leftTableLoader = - TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("left")); - - Table rightTable = - CATALOG_EXTENSION - .catalog() - .createTable( - TableIdentifier.of("right"), - SimpleDataUtil.SCHEMA, - partitioned - ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() - : PartitionSpec.unpartitioned(), - props); - TableLoader rightTableLoader = - TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("right")); - - env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) - .enableCheckpointing(100) - .setParallelism(parallelism) - .setMaxParallelism(parallelism); - env.getConfig().disableAutoGeneratedUIDs(); - - List leftRows = createRows("left-"); - DataStream leftStream = - env.fromCollection(leftRows, ROW_TYPE_INFO) - .name("leftCustomSource") - .uid("leftCustomSource"); - FlinkSink.forRow(leftStream, SimpleDataUtil.FLINK_SCHEMA) - .table(leftTable) - .tableLoader(leftTableLoader) - .tableSchema(SimpleDataUtil.FLINK_SCHEMA) - .distributionMode(DistributionMode.NONE) - .uidPrefix("leftIcebergSink") - .append(); - - List rightRows = createRows("right-"); - DataStream rightStream = - env.fromCollection(rightRows, ROW_TYPE_INFO) - .name("rightCustomSource") - .uid("rightCustomSource"); - FlinkSink.forRow(rightStream, SimpleDataUtil.FLINK_SCHEMA) - .table(rightTable) - .tableLoader(rightTableLoader) - .tableSchema(SimpleDataUtil.FLINK_SCHEMA) - .writeParallelism(parallelism) - .distributionMode(DistributionMode.HASH) - .uidPrefix("rightIcebergSink") - .setSnapshotProperty("flink.test", TestFlinkIcebergSink.class.getName()) - .setSnapshotProperties(Collections.singletonMap("direction", "rightTable")) - .append(); - - // Execute the program. - env.execute("Test Iceberg DataStream."); - - SimpleDataUtil.assertTableRows(leftTable, convertToRowData(leftRows)); - SimpleDataUtil.assertTableRows(rightTable, convertToRowData(rightRows)); - - leftTable.refresh(); - assertThat(leftTable.currentSnapshot().summary()).doesNotContainKeys("flink.test", "direction"); - rightTable.refresh(); - assertThat(rightTable.currentSnapshot().summary()) - .containsEntry("flink.test", TestFlinkIcebergSink.class.getName()) - .containsEntry("direction", "rightTable"); - } - - @TestTemplate - public void testOverrideWriteConfigWithUnknownDistributionMode() { - Map newProps = Maps.newHashMap(); - newProps.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), "UNRECOGNIZED"); - - List rows = createRows(""); - DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); - - FlinkSink.Builder builder = - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .table(table) - .tableLoader(tableLoader) - .writeParallelism(parallelism) - .setAll(newProps); - - assertThatThrownBy(builder::append) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid distribution mode: UNRECOGNIZED"); - } - - @TestTemplate - public void testOverrideWriteConfigWithUnknownFileFormat() { - Map newProps = Maps.newHashMap(); - newProps.put(FlinkWriteOptions.WRITE_FORMAT.key(), "UNRECOGNIZED"); - - List rows = createRows(""); - DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); - - FlinkSink.Builder builder = - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .table(table) - .tableLoader(tableLoader) - .writeParallelism(parallelism) - .setAll(newProps); - - assertThatThrownBy(builder::append) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid file format: UNRECOGNIZED"); - } - - @TestTemplate - public void testWriteRowWithTableRefreshInterval() throws Exception { - List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); - DataStream dataStream = - env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) - .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); - - Configuration flinkConf = new Configuration(); - flinkConf.setString(FlinkWriteOptions.TABLE_REFRESH_INTERVAL.key(), "100ms"); - - FlinkSink.forRowData(dataStream) - .table(table) - .tableLoader(tableLoader) - .flinkConf(flinkConf) - .writeParallelism(parallelism) - .append(); - - // Execute the program. - env.execute("Test Iceberg DataStream"); - - // Assert the iceberg table's records. - SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + testWriteRow(parallelism, SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java index b38aa6b50ce6..9ce36cc1e8d0 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java @@ -18,29 +18,52 @@ */ package org.apache.iceberg.flink.sink; +import static org.apache.iceberg.flink.TestFixtures.DATABASE; + +import java.io.IOException; import java.util.List; import java.util.stream.Collectors; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.util.DataFormatConverters; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; import org.apache.iceberg.Table; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.source.BoundedTestSource; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.extension.RegisterExtension; public class TestFlinkIcebergSinkBase { - protected Table table; - protected StreamExecutionEnvironment env; + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @RegisterExtension + protected static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); + protected static final TypeInformation ROW_TYPE_INFO = new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); protected static final DataFormatConverters.RowConverter CONVERTER = new DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes()); + protected TableLoader tableLoader; + protected Table table; + protected StreamExecutionEnvironment env; + protected BoundedTestSource createBoundedSource(List rows) { return new BoundedTestSource<>(rows.toArray(new Row[0])); } @@ -61,4 +84,28 @@ protected List createRows(String prefix) { protected List convertToRowData(List rows) { return rows.stream().map(CONVERTER::toInternal).collect(Collectors.toList()); } + + protected void testWriteRow( + int writerParallelism, TableSchema tableSchema, DistributionMode distributionMode) + throws Exception { + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .writeParallelism(writerParallelism) + .distributionMode(distributionMode) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } + + protected int partitionFiles(String partition) throws IOException { + return SimpleDataUtil.partitionDataFiles(table, ImmutableMap.of("data", partition)).size(); + } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java new file mode 100644 index 000000000000..df8c3c79d3e3 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java @@ -0,0 +1,494 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.sink.shuffle.StatisticsType; +import org.apache.iceberg.flink.source.BoundedTestSource; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; + +/** + * This tests the distribution mode of Flink sink. Extract them separately since it is unnecessary + * to test different file formats (Avro, Orc, Parquet) like in {@link TestFlinkIcebergSink}. + * Removing the file format dimension reduces the number of combinations from 12 to 4, which helps + * reduce test run time. + */ +@ExtendWith(ParameterizedTestExtension.class) +public class TestFlinkIcebergSinkDistributionMode extends TestFlinkIcebergSinkBase { + + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + private final FileFormat format = FileFormat.PARQUET; + + @Parameter(index = 0) + private int parallelism; + + @Parameter(index = 1) + private boolean partitioned; + + @Parameters(name = "parallelism = {0}, partitioned = {1}") + public static Object[][] parameters() { + return new Object[][] { + {1, true}, + {1, false}, + {2, true}, + {2, false} + }; + } + + @BeforeEach + public void before() throws IOException { + this.table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); + + this.env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + this.tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testShuffleByPartitionWithSchema() throws Exception { + testWriteRow(parallelism, SimpleDataUtil.FLINK_SCHEMA, DistributionMode.HASH); + if (partitioned) { + assertThat(partitionFiles("aaa")).isEqualTo(1); + assertThat(partitionFiles("bbb")).isEqualTo(1); + assertThat(partitionFiles("ccc")).isEqualTo(1); + } + } + + @TestTemplate + public void testJobNoneDistributeMode() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + testWriteRow(parallelism, null, DistributionMode.NONE); + + if (parallelism > 1) { + if (partitioned) { + int files = partitionFiles("aaa") + partitionFiles("bbb") + partitionFiles("ccc"); + assertThat(files).isGreaterThan(3); + } + } + } + + @TestTemplate + public void testJobNullDistributionMode() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + testWriteRow(parallelism, null, null); + + if (partitioned) { + assertThat(partitionFiles("aaa")).isEqualTo(1); + assertThat(partitionFiles("bbb")).isEqualTo(1); + assertThat(partitionFiles("ccc")).isEqualTo(1); + } + } + + @TestTemplate + public void testPartitionWriteMode() throws Exception { + testWriteRow(parallelism, null, DistributionMode.HASH); + if (partitioned) { + assertThat(partitionFiles("aaa")).isEqualTo(1); + assertThat(partitionFiles("bbb")).isEqualTo(1); + assertThat(partitionFiles("ccc")).isEqualTo(1); + } + } + + @TestTemplate + public void testOverrideWriteConfigWithUnknownDistributionMode() { + Map newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), "UNRECOGNIZED"); + + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .setAll(newProps); + + assertThatThrownBy(builder::append) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid distribution mode: UNRECOGNIZED"); + } + + @TestTemplate + public void testRangeDistributionWithoutSortOrderUnpartitioned() throws Exception { + assumeThat(partitioned).isFalse(); + + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + + int numOfCheckpoints = 6; + DataStream dataStream = + env.addSource( + createRangeDistributionBoundedSource(createCharRows(numOfCheckpoints, 10)), + ROW_TYPE_INFO); + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism); + + // Range distribution requires either sort order or partition spec defined + assertThatThrownBy(builder::append) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Invalid write distribution mode: range. Need to define sort order or partition spec."); + } + + @TestTemplate + public void testRangeDistributionWithoutSortOrderPartitioned() throws Exception { + assumeThat(partitioned).isTrue(); + + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + + int numOfCheckpoints = 6; + DataStream dataStream = + env.addSource( + createRangeDistributionBoundedSource(createCharRows(numOfCheckpoints, 10)), + ROW_TYPE_INFO); + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism); + + // sort based on partition columns + builder.append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(numOfCheckpoints); + } + + @TestTemplate + public void testRangeDistributionWithSortOrder() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + table.replaceSortOrder().asc("data").commit(); + + int numOfCheckpoints = 6; + DataStream dataStream = + env.addSource( + createRangeDistributionBoundedSource(createCharRows(numOfCheckpoints, 10)), + ROW_TYPE_INFO); + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .rangeDistributionStatisticsType(StatisticsType.Map) + .append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(numOfCheckpoints); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + if (partitioned) { + for (Snapshot snapshot : rangePartitionedCycles) { + List addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // up to 26 partitions + assertThat(addedDataFiles).hasSizeLessThanOrEqualTo(26); + } + } else { + for (Snapshot snapshot : rangePartitionedCycles) { + List addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // each writer task should only write one file for non-partition sort column + assertThat(addedDataFiles).hasSize(parallelism); + // verify there is no overlap in min-max stats range + if (parallelism == 2) { + assertIdColumnStatsNoRangeOverlap(addedDataFiles.get(0), addedDataFiles.get(1)); + } + } + } + } + + @TestTemplate + public void testRangeDistributionSketchWithSortOrder() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + table.replaceSortOrder().asc("id").commit(); + + int numOfCheckpoints = 6; + DataStream dataStream = + env.addSource( + createRangeDistributionBoundedSource(createIntRows(numOfCheckpoints, 1_000)), + ROW_TYPE_INFO); + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .rangeDistributionStatisticsType(StatisticsType.Sketch) + .append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(numOfCheckpoints); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + // since the input has a single value for the data column, + // it is always the same partition. Hence there is no difference + // for partitioned or not + for (Snapshot snapshot : rangePartitionedCycles) { + List addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // each writer task should only write one file for non-partition sort column + assertThat(addedDataFiles).hasSize(parallelism); + // verify there is no overlap in min-max stats range + if (parallelism == 2) { + assertIdColumnStatsNoRangeOverlap(addedDataFiles.get(0), addedDataFiles.get(1)); + } + } + } + + /** Test migration from Map stats to Sketch stats */ + @TestTemplate + public void testRangeDistributionStatisticsMigration() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + table.replaceSortOrder().asc("id").commit(); + + int numOfCheckpoints = 4; + List> rowsPerCheckpoint = Lists.newArrayListWithCapacity(numOfCheckpoints); + for (int checkpointId = 0; checkpointId < numOfCheckpoints; ++checkpointId) { + // checkpointId 2 would emit 11_000 records which is larger than + // the OPERATOR_SKETCH_SWITCH_THRESHOLD of 10_000. + // This should trigger the stats migration. + int maxId = checkpointId < 1 ? 1_000 : 11_000; + List rows = Lists.newArrayListWithCapacity(maxId); + for (int j = 0; j < maxId; ++j) { + // fixed value "a" for the data (possible partition column) + rows.add(Row.of(j, "a")); + } + + rowsPerCheckpoint.add(rows); + } + + DataStream dataStream = + env.addSource(createRangeDistributionBoundedSource(rowsPerCheckpoint), ROW_TYPE_INFO); + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .rangeDistributionStatisticsType(StatisticsType.Auto) + .append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(numOfCheckpoints); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + // since the input has a single value for the data column, + // it is always the same partition. Hence there is no difference + // for partitioned or not + for (Snapshot snapshot : rangePartitionedCycles) { + List addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // each writer task should only write one file for non-partition sort column + // sometimes + assertThat(addedDataFiles).hasSize(parallelism); + // verify there is no overlap in min-max stats range + if (parallelism == 2) { + assertIdColumnStatsNoRangeOverlap(addedDataFiles.get(0), addedDataFiles.get(1)); + } + } + } + + private BoundedTestSource createRangeDistributionBoundedSource( + List> rowsPerCheckpoint) { + return new BoundedTestSource<>(rowsPerCheckpoint); + } + + private List> createCharRows(int numOfCheckpoints, int countPerChar) { + List> rowsPerCheckpoint = Lists.newArrayListWithCapacity(numOfCheckpoints); + for (int checkpointId = 0; checkpointId < numOfCheckpoints; ++checkpointId) { + List rows = Lists.newArrayListWithCapacity(26 * countPerChar); + for (int j = 0; j < countPerChar; ++j) { + for (char c = 'a'; c <= 'z'; ++c) { + rows.add(Row.of(1, String.valueOf(c))); + } + } + + rowsPerCheckpoint.add(rows); + } + + return rowsPerCheckpoint; + } + + private List> createIntRows(int numOfCheckpoints, int maxId) { + List> rowsPerCheckpoint = Lists.newArrayListWithCapacity(numOfCheckpoints); + for (int checkpointId = 0; checkpointId < numOfCheckpoints; ++checkpointId) { + List rows = Lists.newArrayListWithCapacity(maxId); + for (int j = 0; j < maxId; ++j) { + // fixed value "a" for the data (possible partition column) + rows.add(Row.of(j, "a")); + } + + rowsPerCheckpoint.add(rows); + } + + return rowsPerCheckpoint; + } + + private void assertIdColumnStatsNoRangeOverlap(DataFile file1, DataFile file2) { + // id column has fieldId 1 + int file1LowerBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file1.lowerBounds().get(1)); + int file1UpperBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file1.upperBounds().get(1)); + int file2LowerBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file2.lowerBounds().get(1)); + int file2UpperBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file2.upperBounds().get(1)); + + if (file1LowerBound < file2LowerBound) { + assertThat(file1UpperBound).isLessThanOrEqualTo(file2LowerBound); + } else { + assertThat(file2UpperBound).isLessThanOrEqualTo(file1LowerBound); + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java new file mode 100644 index 000000000000..36a59b20431c --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java @@ -0,0 +1,208 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +/** + * This class tests the more extended features of Flink sink. Extract them separately since it is + * unnecessary to test all the parameters combinations in {@link TestFlinkIcebergSink}. Each test + * method in {@link TestFlinkIcebergSink} runs 12 combinations, which are expensive and slow. + */ +public class TestFlinkIcebergSinkExtended extends TestFlinkIcebergSinkBase { + private final boolean partitioned = true; + private final int parallelism = 2; + private final FileFormat format = FileFormat.PARQUET; + + @BeforeEach + public void before() throws IOException { + this.table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); + + this.env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + this.tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @Test + public void testTwoSinksInDisjointedDAG() throws Exception { + Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); + + Table leftTable = + CATALOG_EXTENSION + .catalog() + .createTable( + TableIdentifier.of("left"), + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + props); + TableLoader leftTableLoader = + TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("left")); + + Table rightTable = + CATALOG_EXTENSION + .catalog() + .createTable( + TableIdentifier.of("right"), + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + props); + TableLoader rightTableLoader = + TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("right")); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + env.getConfig().disableAutoGeneratedUIDs(); + + List leftRows = createRows("left-"); + DataStream leftStream = + env.fromCollection(leftRows, ROW_TYPE_INFO) + .name("leftCustomSource") + .uid("leftCustomSource"); + FlinkSink.forRow(leftStream, SimpleDataUtil.FLINK_SCHEMA) + .table(leftTable) + .tableLoader(leftTableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .distributionMode(DistributionMode.NONE) + .uidPrefix("leftIcebergSink") + .append(); + + List rightRows = createRows("right-"); + DataStream rightStream = + env.fromCollection(rightRows, ROW_TYPE_INFO) + .name("rightCustomSource") + .uid("rightCustomSource"); + FlinkSink.forRow(rightStream, SimpleDataUtil.FLINK_SCHEMA) + .table(rightTable) + .tableLoader(rightTableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .distributionMode(DistributionMode.HASH) + .uidPrefix("rightIcebergSink") + .setSnapshotProperty("flink.test", TestFlinkIcebergSink.class.getName()) + .setSnapshotProperties(Collections.singletonMap("direction", "rightTable")) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(leftTable, convertToRowData(leftRows)); + SimpleDataUtil.assertTableRows(rightTable, convertToRowData(rightRows)); + + leftTable.refresh(); + assertThat(leftTable.currentSnapshot().summary()).doesNotContainKeys("flink.test", "direction"); + rightTable.refresh(); + assertThat(rightTable.currentSnapshot().summary()) + .containsEntry("flink.test", TestFlinkIcebergSink.class.getName()) + .containsEntry("direction", "rightTable"); + } + + @Test + public void testOverrideWriteConfigWithUnknownFileFormat() { + Map newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.WRITE_FORMAT.key(), "UNRECOGNIZED"); + + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .setAll(newProps); + + assertThatThrownBy(builder::append) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid file format: UNRECOGNIZED"); + } + + @Test + public void testWriteRowWithTableRefreshInterval() throws Exception { + List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) + .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); + + Configuration flinkConf = new Configuration(); + flinkConf.setString(FlinkWriteOptions.TABLE_REFRESH_INTERVAL.key(), "100ms"); + + FlinkSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .flinkConf(flinkConf) + .writeParallelism(parallelism) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream"); + + // Assert the iceberg table's records. + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java index 577c54976b9a..b283b8390a2b 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -30,6 +30,7 @@ import org.apache.flink.types.Row; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.DistributionMode; import org.apache.iceberg.FileFormat; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.ParameterizedTestExtension; @@ -184,11 +185,21 @@ public void testUpsertModeCheck() throws Exception { .hasMessage( "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); - assertThatThrownBy( - () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) - .isInstanceOf(IllegalStateException.class) - .hasMessage( - "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + if (writeDistributionMode.equals(DistributionMode.RANGE.modeName()) && !partitioned) { + // validation error thrown from distributeDataStream + assertThatThrownBy( + () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Invalid write distribution mode: range. Need to define sort order or partition spec."); + } else { + // validation error thrown from appendWriter + assertThatThrownBy( + () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + } } @TestTemplate diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java index 0b0c55f51c32..52826305b4c6 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java @@ -46,12 +46,21 @@ public class TestFlinkIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Base { private static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); - @Parameter(index = 0) - private String branch; + @Parameter(index = 4) + protected String branch; - @Parameters(name = "branch = {0}") + @Parameters( + name = + "FileFormat={0}, Parallelism={1}, Partitioned={2}, WriteDistributionMode={3}, Branch={4}") public static Object[][] parameters() { - return new Object[][] {new Object[] {"main"}, new Object[] {"testBranch"}}; + return new Object[][] { + new Object[] { + FileFormat.AVRO, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE, "main" + }, + new Object[] { + FileFormat.AVRO, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE, "testBranch" + } + }; } @BeforeEach diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java index 948c7b31430c..ac5babe11943 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java @@ -129,7 +129,8 @@ public void testCommitTxnWithoutDataFiles() throws Exception { long timestamp = 0; JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -161,7 +162,8 @@ public void testMaxContinuousEmptyCommits() throws Exception { JobID jobId = new JobID(); long checkpointId = 0; long timestamp = 0; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); @@ -176,8 +178,8 @@ public void testMaxContinuousEmptyCommits() throws Exception { } } - private WriteResult of(DataFile dataFile) { - return WriteResult.builder().addDataFiles(dataFile).build(); + private FlinkWriteResult of(long checkpointId, DataFile dataFile) { + return new FlinkWriteResult(checkpointId, WriteResult.builder().addDataFiles(dataFile).build()); } @TestTemplate @@ -193,7 +195,8 @@ public void testCommitTxn() throws Exception { JobID jobID = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobID)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobID)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -204,7 +207,7 @@ public void testCommitTxn() throws Exception { for (int i = 1; i <= 3; i++) { RowData rowData = SimpleDataUtil.createRowData(i, "hello" + i); DataFile dataFile = writeDataFile("data-" + i, ImmutableList.of(rowData)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(i, dataFile), ++timestamp); rows.add(rowData); harness.snapshot(i, ++timestamp); @@ -233,7 +236,8 @@ public void testOrderedEventsBetweenCheckpoints() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -243,21 +247,21 @@ public void testOrderedEventsBetweenCheckpoints() throws Exception { RowData row1 = SimpleDataUtil.createRowData(1, "hello"); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + long firstCheckpointId = 1; + harness.processElement(of(firstCheckpointId, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 - long firstCheckpointId = 1; harness.snapshot(firstCheckpointId, ++timestamp); assertFlinkManifests(1); RowData row2 = SimpleDataUtil.createRowData(2, "world"); DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); - harness.processElement(of(dataFile2), ++timestamp); + long secondCheckpointId = 2; + harness.processElement(of(secondCheckpointId, dataFile2), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 2. snapshotState for checkpoint#2 - long secondCheckpointId = 2; harness.snapshot(secondCheckpointId, ++timestamp); assertFlinkManifests(2); @@ -286,7 +290,8 @@ public void testDisorderedEventsBetweenCheckpoints() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -296,21 +301,21 @@ public void testDisorderedEventsBetweenCheckpoints() throws Exception { RowData row1 = SimpleDataUtil.createRowData(1, "hello"); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + long firstCheckpointId = 1; + harness.processElement(of(firstCheckpointId, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 - long firstCheckpointId = 1; harness.snapshot(firstCheckpointId, ++timestamp); assertFlinkManifests(1); RowData row2 = SimpleDataUtil.createRowData(2, "world"); DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); - harness.processElement(of(dataFile2), ++timestamp); + long secondCheckpointId = 2; + harness.processElement(of(secondCheckpointId, dataFile2), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 2. snapshotState for checkpoint#2 - long secondCheckpointId = 2; harness.snapshot(secondCheckpointId, ++timestamp); assertFlinkManifests(2); @@ -337,7 +342,8 @@ public void testRecoveryFromValidSnapshot() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -349,8 +355,8 @@ public void testRecoveryFromValidSnapshot() throws Exception { expectedRows.add(row); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row)); - harness.processElement(of(dataFile1), ++timestamp); - snapshot = harness.snapshot(++checkpointId, ++timestamp); + harness.processElement(of(++checkpointId, dataFile1), ++timestamp); + snapshot = harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -362,7 +368,8 @@ public void testRecoveryFromValidSnapshot() throws Exception { } // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.initializeState(snapshot); @@ -375,9 +382,9 @@ public void testRecoveryFromValidSnapshot() throws Exception { RowData row = SimpleDataUtil.createRowData(2, "world"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -400,7 +407,8 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except List expectedRows = Lists.newArrayList(); JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -411,15 +419,16 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(1, "hello"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-1", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - snapshot = harness.snapshot(++checkpointId, ++timestamp); + snapshot = harness.snapshot(checkpointId, ++timestamp); SimpleDataUtil.assertTableRows(table, ImmutableList.of(), branch); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); assertFlinkManifests(1); } - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.initializeState(snapshot); @@ -446,15 +455,15 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(2, "world"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - snapshot = harness.snapshot(++checkpointId, ++timestamp); + snapshot = harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); } // Redeploying flink job from external checkpoint. JobID newJobId = new JobID(); - try (OneInputStreamOperatorTestHarness harness = + try (OneInputStreamOperatorTestHarness harness = createStreamSink(newJobId)) { harness.setup(); harness.initializeState(snapshot); @@ -473,9 +482,9 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(3, "foo"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-3", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -496,7 +505,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { JobID oldJobId = new JobID(); OperatorID oldOperatorId; - try (OneInputStreamOperatorTestHarness harness = + try (OneInputStreamOperatorTestHarness harness = createStreamSink(oldJobId)) { harness.setup(); harness.open(); @@ -510,8 +519,8 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { tableRows.addAll(rows); DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); - harness.processElement(of(dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -528,7 +537,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { timestamp = 0; JobID newJobId = new JobID(); OperatorID newOperatorId; - try (OneInputStreamOperatorTestHarness harness = + try (OneInputStreamOperatorTestHarness harness = createStreamSink(newJobId)) { harness.setup(); harness.open(); @@ -542,8 +551,8 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { tableRows.addAll(rows); DataFile dataFile = writeDataFile("data-new-1", rows); - harness.processElement(of(dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -567,7 +576,8 @@ public void testMultipleJobsWriteSameTable() throws Exception { int checkpointId = i / 3; JobID jobId = jobs[jobIndex]; OperatorID operatorId = operatorIds[jobIndex]; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.open(); @@ -579,7 +589,7 @@ public void testMultipleJobsWriteSameTable() throws Exception { tableRows.addAll(rows); DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId + 1, dataFile), ++timestamp); harness.snapshot(checkpointId + 1, ++timestamp); assertFlinkManifests(1); @@ -603,8 +613,10 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { JobID jobId = new JobID(); OperatorID operatorId1 = new OperatorID(); OperatorID operatorId2 = new OperatorID(); - try (OneInputStreamOperatorTestHarness harness1 = createStreamSink(jobId); - OneInputStreamOperatorTestHarness harness2 = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness1 = + createStreamSink(jobId); + OneInputStreamOperatorTestHarness harness2 = + createStreamSink(jobId)) { harness1.getStreamConfig().setOperatorID(operatorId1); harness1.setup(); harness1.open(); @@ -620,14 +632,14 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { expectedRows.add(row1); DataFile dataFile1 = writeDataFile("data-1-1", ImmutableList.of(row1)); - harness1.processElement(of(dataFile1), ++timestamp); - snapshot1 = harness1.snapshot(++checkpointId, ++timestamp); + harness1.processElement(of(++checkpointId, dataFile1), ++timestamp); + snapshot1 = harness1.snapshot(checkpointId, ++timestamp); RowData row2 = SimpleDataUtil.createRowData(1, "hello2"); expectedRows.add(row2); DataFile dataFile2 = writeDataFile("data-1-2", ImmutableList.of(row2)); - harness2.processElement(of(dataFile2), ++timestamp); + harness2.processElement(of(checkpointId, dataFile2), ++timestamp); snapshot2 = harness2.snapshot(checkpointId, ++timestamp); assertFlinkManifests(2); @@ -643,8 +655,10 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { } // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness harness1 = createStreamSink(jobId); - OneInputStreamOperatorTestHarness harness2 = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness1 = + createStreamSink(jobId); + OneInputStreamOperatorTestHarness harness2 = + createStreamSink(jobId)) { harness1.getStreamConfig().setOperatorID(operatorId1); harness1.setup(); harness1.initializeState(snapshot1); @@ -668,13 +682,13 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { expectedRows.add(row1); DataFile dataFile1 = writeDataFile("data-2-1", ImmutableList.of(row1)); - harness1.processElement(of(dataFile1), ++timestamp); - harness1.snapshot(++checkpointId, ++timestamp); + harness1.processElement(of(++checkpointId, dataFile1), ++timestamp); + harness1.snapshot(checkpointId, ++timestamp); RowData row2 = SimpleDataUtil.createRowData(2, "world2"); expectedRows.add(row2); DataFile dataFile2 = writeDataFile("data-2-2", ImmutableList.of(row2)); - harness2.processElement(of(dataFile2), ++timestamp); + harness2.processElement(of(checkpointId, dataFile2), ++timestamp); harness2.snapshot(checkpointId, ++timestamp); assertFlinkManifests(2); @@ -694,7 +708,8 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { public void testBoundedStream() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -706,13 +721,14 @@ public void testBoundedStream() throws Exception { List tableRows = Lists.newArrayList(SimpleDataUtil.createRowData(1, "word-1")); DataFile dataFile = writeDataFile("data-1", tableRows); - harness.processElement(of(dataFile), 1); + harness.processElement(of(IcebergStreamWriter.END_INPUT_CHECKPOINT_ID, dataFile), 1); ((BoundedOneInput) harness.getOneInputOperator()).endInput(); assertFlinkManifests(0); SimpleDataUtil.assertTableRows(table, tableRows, branch); assertSnapshotSize(1); - assertMaxCommittedCheckpointId(jobId, operatorId, Long.MAX_VALUE); + assertMaxCommittedCheckpointId( + jobId, operatorId, IcebergStreamWriter.END_INPUT_CHECKPOINT_ID); assertThat(SimpleDataUtil.latestSnapshot(table, branch).summary()) .containsEntry("flink.test", TestIcebergFilesCommitter.class.getName()); } @@ -725,7 +741,8 @@ public void testFlinkManifests() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -735,7 +752,7 @@ public void testFlinkManifests() throws Exception { RowData row1 = SimpleDataUtil.createRowData(1, "hello"); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + harness.processElement(of(checkpoint, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 @@ -775,7 +792,8 @@ public void testDeleteFiles() throws Exception { OperatorID operatorId; FileAppenderFactory appenderFactory = createDeletableAppenderFactory(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -784,7 +802,7 @@ public void testDeleteFiles() throws Exception { RowData row1 = SimpleDataUtil.createInsert(1, "aaa"); DataFile dataFile1 = writeDataFile("data-file-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + harness.processElement(of(checkpoint, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 @@ -816,13 +834,15 @@ public void testDeleteFiles() throws Exception { RowData delete1 = SimpleDataUtil.createDelete(1, "aaa"); DeleteFile deleteFile1 = writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete1)); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); harness.processElement( - WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile1).build(), + new FlinkWriteResult( + ++checkpoint, + WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile1).build()), ++timestamp); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); // 5. snapshotState for checkpoint#2 - harness.snapshot(++checkpoint, ++timestamp); + harness.snapshot(checkpoint, ++timestamp); assertFlinkManifests(2); // 6. notifyCheckpointComplete for checkpoint#2 @@ -846,7 +866,8 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { OperatorID operatorId; FileAppenderFactory appenderFactory = createDeletableAppenderFactory(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -860,7 +881,9 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { DeleteFile deleteFile1 = writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete3)); harness.processElement( - WriteResult.builder().addDataFiles(dataFile1).addDeleteFiles(deleteFile1).build(), + new FlinkWriteResult( + checkpoint, + WriteResult.builder().addDataFiles(dataFile1).addDeleteFiles(deleteFile1).build()), ++timestamp); // The 1th snapshotState. @@ -872,11 +895,13 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { DeleteFile deleteFile2 = writeEqDeleteFile(appenderFactory, "delete-file-2", ImmutableList.of(delete2)); harness.processElement( - WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile2).build(), + new FlinkWriteResult( + ++checkpoint, + WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile2).build()), ++timestamp); // The 2nd snapshotState. - harness.snapshot(++checkpoint, ++timestamp); + harness.snapshot(checkpoint, ++timestamp); // Notify the 2nd snapshot to complete. harness.notifyOfCompletedCheckpoint(checkpoint); @@ -887,6 +912,79 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { } } + /** + * The testcase is to simulate upserting to an Iceberg V2 table, and facing the following + * scenario: + * + *

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

Previously the files from the 2 snapshots were committed in a single Iceberg commit, as a + * results duplicate rows were created in the table. + * + * @throws Exception Exception + */ + @TestTemplate + public void testCommitMultipleCheckpointsForV2Table() throws Exception { + assumeThat(formatVersion) + .as("Only support equality-delete in format v2 or later.") + .isGreaterThan(1); + + long timestamp = 0; + long checkpoint = 10; + + JobID jobId = new JobID(); + OperatorID operatorId; + + FileAppenderFactory appenderFactory = + new FlinkAppenderFactory( + table, + table.schema(), + FlinkSchemaUtil.convert(table.schema()), + table.properties(), + table.spec(), + new int[] {table.schema().findField("id").fieldId()}, + table.schema(), + null); + + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + RowData insert1 = null; + RowData insert2 = null; + for (int i = 1; i <= 3; i++) { + insert1 = SimpleDataUtil.createInsert(1, "aaa" + i); + insert2 = SimpleDataUtil.createInsert(2, "bbb" + i); + DataFile dataFile = writeDataFile("data-file-" + i, ImmutableList.of(insert1, insert2)); + DeleteFile deleteFile = + writeEqDeleteFile( + appenderFactory, "delete-file-" + i, ImmutableList.of(insert1, insert2)); + harness.processElement( + new FlinkWriteResult( + ++checkpoint, + WriteResult.builder().addDataFiles(dataFile).addDeleteFiles(deleteFile).build()), + ++timestamp); + } + + harness.snapshot(checkpoint, ++timestamp); + harness.notifyOfCompletedCheckpoint(checkpoint); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(insert1, insert2), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); + assertFlinkManifests(0); + assertThat(table.snapshots()).hasSize(3); + } + } + @TestTemplate public void testSpecEvolution() throws Exception { long timestamp = 0; @@ -899,7 +997,8 @@ public void testSpecEvolution() throws Exception { DataFile dataFile; int specId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -910,7 +1009,7 @@ public void testSpecEvolution() throws Exception { RowData rowData = SimpleDataUtil.createRowData(checkpointId, "hello" + checkpointId); // table unpartitioned dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(rowData)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId, dataFile), ++timestamp); rows.add(rowData); harness.snapshot(checkpointId, ++timestamp); @@ -929,7 +1028,7 @@ public void testSpecEvolution() throws Exception { rowData = SimpleDataUtil.createRowData(checkpointId, "hello" + checkpointId); // write data with old partition spec dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(rowData), oldSpec, null); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId, dataFile), ++timestamp); rows.add(rowData); snapshot = harness.snapshot(checkpointId, ++timestamp); @@ -947,7 +1046,8 @@ public void testSpecEvolution() throws Exception { } // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.initializeState(snapshot); @@ -963,7 +1063,7 @@ public void testSpecEvolution() throws Exception { partition.set(0, checkpointId); dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(row), table.spec(), partition); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId, dataFile), ++timestamp); rows.add(row); harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); @@ -1089,7 +1189,7 @@ private void assertSnapshotSize(int expectedSnapshotSize) { assertThat(table.snapshots()).hasSize(expectedSnapshotSize); } - private OneInputStreamOperatorTestHarness createStreamSink(JobID jobID) + private OneInputStreamOperatorTestHarness createStreamSink(JobID jobID) throws Exception { TestOperatorFactory factory = TestOperatorFactory.of(table.location(), branch, table.spec()); return new OneInputStreamOperatorTestHarness<>(factory, createEnvironment(jobID)); @@ -1109,7 +1209,7 @@ private static MockEnvironment createEnvironment(JobID jobID) { } private static class TestOperatorFactory extends AbstractStreamOperatorFactory - implements OneInputStreamOperatorFactory { + implements OneInputStreamOperatorFactory { private final String tablePath; private final String branch; private final PartitionSpec spec; diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java index 50283f7ad215..e13721a9f170 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java @@ -28,6 +28,7 @@ import java.util.Locale; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; import org.apache.flink.streaming.api.operators.BoundedOneInput; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.table.api.DataTypes; @@ -102,7 +103,7 @@ public void before() throws IOException { @TestTemplate public void testWritingTable() throws Exception { long checkpointId = 1L; - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { // The first checkpoint testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); @@ -111,7 +112,8 @@ public void testWritingTable() throws Exception { testHarness.prepareSnapshotPreBarrier(checkpointId); int expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); @@ -123,7 +125,8 @@ public void testWritingTable() throws Exception { testHarness.prepareSnapshotPreBarrier(checkpointId); expectedDataFiles = partitioned ? 4 : 2; - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); @@ -148,14 +151,15 @@ public void testWritingTable() throws Exception { public void testSnapshotTwice() throws Exception { long checkpointId = 1; long timestamp = 1; - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), timestamp++); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), timestamp); testHarness.prepareSnapshotPreBarrier(checkpointId++); int expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); @@ -163,7 +167,10 @@ public void testSnapshotTwice() throws Exception { for (int i = 0; i < 5; i++) { testHarness.prepareSnapshotPreBarrier(checkpointId++); - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + result = + WriteResult.builder() + .addAll(getWriteResults(testHarness.extractOutputValues())) + .build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); } @@ -172,14 +179,14 @@ public void testSnapshotTwice() throws Exception { @TestTemplate public void testTableWithoutSnapshot() throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { assertThat(testHarness.extractOutputValues()).isEmpty(); } // Even if we closed the iceberg stream writer, there's no orphan data file. assertThat(scanDataFiles()).isEmpty(); - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); // Still not emit the data file yet, because there is no checkpoint. @@ -212,7 +219,7 @@ private Set scanDataFiles() throws IOException { @TestTemplate public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); @@ -221,13 +228,15 @@ public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); int expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); // Datafiles should not be sent again assertThat(result.dataFiles()).hasSize(expectedDataFiles); @@ -236,7 +245,7 @@ public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { @TestTemplate public void testBoundedStreamTriggeredEndInputBeforeTriggeringCheckpoint() throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); @@ -244,13 +253,15 @@ public void testBoundedStreamTriggeredEndInputBeforeTriggeringCheckpoint() throw testHarness.endInput(); int expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); testHarness.prepareSnapshotPreBarrier(1L); - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); // It should be ensured that after endInput is triggered, when prepareSnapshotPreBarrier // is triggered, write should only send WriteResult once @@ -275,7 +286,7 @@ public void testTableWithTargetFileSize() throws Exception { } } - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { for (RowData row : rows) { testHarness.processElement(row, 1); @@ -283,7 +294,8 @@ public void testTableWithTargetFileSize() throws Exception { // snapshot the operator. testHarness.prepareSnapshotPreBarrier(1); - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(8); @@ -346,13 +358,14 @@ public void testPromotedFlinkDataType() throws Exception { record.copy(ImmutableMap.of("tinyint", 2, "smallint", 0, "int", 102)), record.copy(ImmutableMap.of("tinyint", 3, "smallint", 32767, "int", 103))); - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter(icebergTable, flinkSchema)) { for (RowData row : rows) { testHarness.processElement(row, 1); } testHarness.prepareSnapshotPreBarrier(1); - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(partitioned ? 3 : 1); @@ -365,12 +378,18 @@ public void testPromotedFlinkDataType() throws Exception { SimpleDataUtil.assertTableRecords(location, expected); } - private OneInputStreamOperatorTestHarness createIcebergStreamWriter() + private static List getWriteResults(List flinkWriteResults) { + return flinkWriteResults.stream() + .map(FlinkWriteResult::writeResult) + .collect(Collectors.toList()); + } + + private OneInputStreamOperatorTestHarness createIcebergStreamWriter() throws Exception { return createIcebergStreamWriter(table, SimpleDataUtil.FLINK_SCHEMA); } - private OneInputStreamOperatorTestHarness createIcebergStreamWriter( + private OneInputStreamOperatorTestHarness createIcebergStreamWriter( Table icebergTable, TableSchema flinkSchema) throws Exception { RowType flinkRowType = FlinkSink.toFlinkRowType(icebergTable.schema(), flinkSchema); FlinkWriteConf flinkWriteConfig = @@ -379,7 +398,7 @@ private OneInputStreamOperatorTestHarness createIcebergStr IcebergStreamWriter streamWriter = FlinkSink.createStreamWriter(() -> icebergTable, flinkWriteConfig, flinkRowType, null); - OneInputStreamOperatorTestHarness harness = + OneInputStreamOperatorTestHarness harness = new OneInputStreamOperatorTestHarness<>(streamWriter, 1, 1, 0); harness.setup(); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java index d5a0bebc74e7..c95a9955b104 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java @@ -380,7 +380,9 @@ private static Map>> runPartitioner( return partitionResults; } - /** @param expectedAssignmentInfo excluding closing cost */ + /** + * @param expectedAssignmentInfo excluding closing cost + */ private void validatePartitionResults( Map> expectedAssignmentInfo, Map>> partitionResults, diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java index e1162c3225b1..2dc5bc5c658e 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java @@ -33,10 +33,12 @@ import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; /** Use the FlinkSource */ public class TestFlinkSourceSql extends TestSqlBase { + @BeforeEach @Override public void before() throws IOException { SqlHelpers.sql( diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java index b7447d15c05a..db8647f054ae 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java @@ -24,8 +24,6 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -130,11 +128,8 @@ protected List run( sourceBuilder.properties(options); DataStream stream = - env.fromSource( - sourceBuilder.build(), - WatermarkStrategy.noWatermarks(), - "testBasicRead", - TypeInformation.of(RowData.class)) + sourceBuilder + .buildStream(env) .map( new RowDataToRowMapper( FlinkSchemaUtil.convert( diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java index 7bfed00a9eb4..4e649d15b1ce 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java @@ -52,6 +52,7 @@ import org.apache.iceberg.flink.data.RowDataToRowMapper; import org.apache.iceberg.flink.sink.AvroGenericRecordToRowDataMapper; import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; +import org.apache.iceberg.flink.source.reader.AvroGenericRecordConverter; import org.apache.iceberg.flink.source.reader.AvroGenericRecordReaderFunction; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; @@ -69,12 +70,13 @@ public class TestIcebergSourceBoundedGenericRecord { private static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); - @Parameters(name = "format={0}, parallelism = {1}") + @Parameters(name = "format={0}, parallelism = {1}, useConverter = {2}") public static Object[][] parameters() { return new Object[][] { - {FileFormat.AVRO, 2}, - {FileFormat.PARQUET, 2}, - {FileFormat.ORC, 2} + {FileFormat.AVRO, 2, true}, + {FileFormat.PARQUET, 2, true}, + {FileFormat.PARQUET, 2, false}, + {FileFormat.ORC, 2, true} }; } @@ -84,6 +86,9 @@ public static Object[][] parameters() { @Parameter(index = 1) private int parallelism; + @Parameter(index = 2) + private boolean useConverter; + @TestTemplate public void testUnpartitionedTable() throws Exception { Table table = @@ -147,24 +152,15 @@ private List run( table = tableLoader.loadTable(); } - AvroGenericRecordReaderFunction readerFunction = - new AvroGenericRecordReaderFunction( - TestFixtures.TABLE_IDENTIFIER.name(), - new Configuration(), - table.schema(), - null, - null, - false, - table.io(), - table.encryption(), - filters); + Schema readSchema = projectedSchema != null ? projectedSchema : table.schema(); + IcebergSource.Builder sourceBuilder; + if (useConverter) { + sourceBuilder = createSourceBuilderWithConverter(table, readSchema, config); + } else { + sourceBuilder = + createSourceBuilderWithReaderFunction(table, projectedSchema, filters, config); + } - IcebergSource.Builder sourceBuilder = - IcebergSource.builder() - .tableLoader(CATALOG_EXTENSION.tableLoader()) - .readerFunction(readerFunction) - .assignerFactory(new SimpleSplitAssignerFactory()) - .flinkConfig(config); if (projectedSchema != null) { sourceBuilder.project(projectedSchema); } @@ -172,7 +168,6 @@ private List run( sourceBuilder.filters(filters); sourceBuilder.setAll(options); - Schema readSchema = projectedSchema != null ? projectedSchema : table.schema(); RowType rowType = FlinkSchemaUtil.convert(readSchema); org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(readSchema, TestFixtures.TABLE_IDENTIFIER.name()); @@ -193,4 +188,35 @@ private List run( return Lists.newArrayList(iter); } } + + private IcebergSource.Builder createSourceBuilderWithReaderFunction( + Table table, Schema projected, List filters, Configuration config) { + AvroGenericRecordReaderFunction readerFunction = + new AvroGenericRecordReaderFunction( + TestFixtures.TABLE_IDENTIFIER.name(), + new Configuration(), + table.schema(), + projected, + null, + false, + table.io(), + table.encryption(), + filters); + + return IcebergSource.builder() + .tableLoader(CATALOG_EXTENSION.tableLoader()) + .readerFunction(readerFunction) + .assignerFactory(new SimpleSplitAssignerFactory()) + .flinkConfig(config); + } + + private IcebergSource.Builder createSourceBuilderWithConverter( + Table table, Schema readSchema, Configuration config) { + AvroGenericRecordConverter converter = + AvroGenericRecordConverter.fromIcebergSchema(readSchema, table.name()); + return IcebergSource.forOutputType(converter) + .tableLoader(CATALOG_EXTENSION.tableLoader()) + .assignerFactory(new SimpleSplitAssignerFactory()) + .flinkConfig(config); + } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java index 0f41c5af4c95..d3713e296014 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java @@ -37,7 +37,7 @@ public class TestIcebergSourceBoundedSql extends TestIcebergSourceBounded { @BeforeEach public void before() throws IOException { Configuration tableConf = getTableEnv().getConfig().getConfiguration(); - tableConf.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), true); + tableConf.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); SqlHelpers.sql( getTableEnv(), "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java new file mode 100644 index 000000000000..2908cb927269 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java @@ -0,0 +1,181 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source; + +import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.nio.file.Path; +import java.util.List; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; +import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.testutils.InternalMiniClusterExtension; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.data.RowDataToRowMapper; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +public class TestIcebergSourceInferParallelism { + private static final int NUM_TMS = 2; + private static final int SLOTS_PER_TM = 2; + private static final int PARALLELISM = NUM_TMS * SLOTS_PER_TM; + private static final int MAX_INFERRED_PARALLELISM = 3; + + @RegisterExtension + private static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(NUM_TMS) + .setNumberSlotsPerTaskManager(SLOTS_PER_TM) + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) + .build()); + + @RegisterExtension + protected static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + @TempDir private Path tmpDir; + + private Table table; + private GenericAppenderHelper dataAppender; + + @BeforeEach + public void before() throws IOException { + this.table = + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + this.dataAppender = new GenericAppenderHelper(table, FileFormat.PARQUET, tmpDir); + } + + @AfterEach + public void after() { + CATALOG_EXTENSION.catalog().dropTable(TestFixtures.TABLE_IDENTIFIER); + } + + @Test + public void testEmptyTable() throws Exception { + // Inferred parallelism should be at least 1 even if table is empty + test(1, 0); + } + + @Test + public void testTableWithFilesLessThanMaxInferredParallelism() throws Exception { + // Append files to the table + for (int i = 0; i < 2; ++i) { + List batch = RandomGenericData.generate(table.schema(), 1, 0); + dataAppender.appendToTable(batch); + } + + // Inferred parallelism should equal to 2 splits + test(2, 2); + } + + @Test + public void testTableWithFilesMoreThanMaxInferredParallelism() throws Exception { + // Append files to the table + for (int i = 0; i < MAX_INFERRED_PARALLELISM + 1; ++i) { + List batch = RandomGenericData.generate(table.schema(), 1, 0); + dataAppender.appendToTable(batch); + } + + // Inferred parallelism should be capped by the MAX_INFERRED_PARALLELISM + test(MAX_INFERRED_PARALLELISM, MAX_INFERRED_PARALLELISM + 1); + } + + private void test(int expectedParallelism, int expectedRecords) throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(PARALLELISM); + + Configuration config = new Configuration(); + config.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, true); + config.set( + FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX, + MAX_INFERRED_PARALLELISM); + + DataStream dataStream = + IcebergSource.forRowData() + .tableLoader(CATALOG_EXTENSION.tableLoader()) + .table(table) + .flinkConfig(config) + // force one file per split + .splitSize(1L) + .buildStream(env) + .map(new RowDataToRowMapper(FlinkSchemaUtil.convert(table.schema()))); + + DataStream.Collector collector = new DataStream.Collector<>(); + dataStream.collectAsync(collector); + JobClient jobClient = env.executeAsync(); + try (CloseableIterator iterator = collector.getOutput()) { + List result = Lists.newArrayList(); + while (iterator.hasNext()) { + result.add(iterator.next()); + } + + assertThat(result).hasSize(expectedRecords); + verifySourceParallelism( + expectedParallelism, miniCluster().getExecutionGraph(jobClient.getJobID()).get()); + } + } + + /** + * Borrowed this approach from Flink {@code FileSourceTextLinesITCase} to get source parallelism + * from execution graph. + */ + private static void verifySourceParallelism( + int expectedParallelism, AccessExecutionGraph executionGraph) { + AccessExecutionJobVertex sourceVertex = + executionGraph.getVerticesTopologically().iterator().next(); + assertThat(sourceVertex.getParallelism()).isEqualTo(expectedParallelism); + } + + /** + * Use reflection to get {@code InternalMiniClusterExtension} and {@code MiniCluster} to get + * execution graph and source parallelism. Haven't find other way via public APIS. + */ + private static MiniCluster miniCluster() throws Exception { + Field privateField = + MiniClusterExtension.class.getDeclaredField("internalMiniClusterExtension"); + privateField.setAccessible(true); + InternalMiniClusterExtension internalExtension = + (InternalMiniClusterExtension) privateField.get(MINI_CLUSTER_EXTENSION); + return internalExtension.getMiniCluster(); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java index 75f0a785a8c5..66bdeee1d407 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java @@ -40,6 +40,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; /** Use the IcebergSource (FLIP-27) */ @@ -49,11 +50,17 @@ public class TestIcebergSourceSql extends TestSqlBase { required(1, "t1", Types.TimestampType.withoutZone()), required(2, "t2", Types.LongType.get())); + @BeforeEach @Override public void before() throws IOException { TableEnvironment tableEnvironment = getTableEnv(); Configuration tableConf = tableEnvironment.getConfig().getConfiguration(); - tableConf.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), true); + tableConf.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); + // Disable inferring parallelism to avoid interfering watermark tests + // that check split assignment is ordered by the watermark column. + // The tests assumes default parallelism of 1 with single reader task + // in order to check the order of read records. + tableConf.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); tableEnvironment.getConfig().set("table.exec.resource.default-parallelism", "1"); SqlHelpers.sql( diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java index 95d0b90b6ca9..564e8139e6cc 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java @@ -48,14 +48,20 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.extension.RegisterExtension; +/** + * There is a infinite sleep in the test. Add a timeout to the test to avoid stuck situation in case + * anything goes wrong unexpectedly. + */ +@Timeout(value = 60) public class TestIcebergSpeculativeExecutionSupport extends TestBase { private static final int NUM_TASK_MANAGERS = 1; private static final int NUM_TASK_SLOTS = 3; @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = new MiniClusterExtension( new MiniClusterResourceConfiguration.Builder() .setNumberTaskManagers(NUM_TASK_MANAGERS) @@ -103,7 +109,7 @@ public void before() throws IOException { public void after() { sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, INPUT_TABLE_NAME); sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, OUTPUT_TABLE_NAME); - sql("DROP DATABASE %s", DATABASE_NAME); + dropDatabase(DATABASE_NAME, true); dropCatalog(CATALOG_NAME, true); } @@ -144,9 +150,9 @@ public void testSpeculativeExecution() throws Exception { private static class TestingMap extends RichMapFunction { @Override public Row map(Row row) throws Exception { - // Put the subtasks with the first attempt to sleep to trigger speculative - // execution - if (getRuntimeContext().getAttemptNumber() <= 0) { + // Simulate slow subtask 0 with attempt 0 + if (getRuntimeContext().getIndexOfThisSubtask() == 0 + && getRuntimeContext().getAttemptNumber() <= 0) { Thread.sleep(Integer.MAX_VALUE); } @@ -169,6 +175,7 @@ private static Configuration configure() { // Use FLIP-27 source configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); + configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); // for speculative execution configuration.set(BatchExecutionOptions.SPECULATIVE_ENABLED, true); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java index 40dfda723749..3aa2a7c0667f 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java @@ -217,27 +217,27 @@ public void testPrimitiveColumns() throws Exception { Row binaryCol = Row.of( - 52L, + 55L, 4L, 2L, null, Base64.getDecoder().decode("1111"), Base64.getDecoder().decode("2222")); - Row booleanCol = Row.of(32L, 4L, 0L, null, false, true); - Row decimalCol = Row.of(85L, 4L, 1L, null, new BigDecimal("1.00"), new BigDecimal("2.00")); - Row doubleCol = Row.of(85L, 4L, 0L, 1L, 1.0D, 2.0D); + Row booleanCol = Row.of(36L, 4L, 0L, null, false, true); + Row decimalCol = Row.of(91L, 4L, 1L, null, new BigDecimal("1.00"), new BigDecimal("2.00")); + Row doubleCol = Row.of(91L, 4L, 0L, 1L, 1.0D, 2.0D); Row fixedCol = Row.of( - 44L, + 47L, 4L, 2L, null, Base64.getDecoder().decode("1111"), Base64.getDecoder().decode("2222")); - Row floatCol = Row.of(71L, 4L, 0L, 2L, 0f, 0f); - Row intCol = Row.of(71L, 4L, 0L, null, 1, 2); - Row longCol = Row.of(79L, 4L, 0L, null, 1L, 2L); - Row stringCol = Row.of(79L, 4L, 0L, null, "1", "2"); + Row floatCol = Row.of(77L, 4L, 0L, 2L, 0f, 0f); + Row intCol = Row.of(77L, 4L, 0L, null, 1, 2); + Row longCol = Row.of(85L, 4L, 0L, null, 1L, 2L); + Row stringCol = Row.of(85L, 4L, 0L, null, "1", "2"); List expected = Lists.newArrayList( @@ -289,8 +289,8 @@ public void testSelectNestedValues() throws Exception { public void testNestedValues() throws Exception { createNestedTable(); - Row leafDoubleCol = Row.of(46L, 3L, 1L, 1L, 0.0D, 0.0D); - Row leafLongCol = Row.of(54L, 3L, 1L, null, 0L, 1L); + Row leafDoubleCol = Row.of(50L, 3L, 1L, 1L, 0.0D, 0.0D); + Row leafLongCol = Row.of(57L, 3L, 1L, null, 0L, 1L); Row metrics = Row.of(Row.of(leafDoubleCol, leafLongCol)); TestHelpers.assertRows( diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java index 0690b456e033..5767fa02c822 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java @@ -80,7 +80,9 @@ private void appendTwoSnapshots() throws IOException { snapshot2 = TABLE_RESOURCE.table().currentSnapshot(); } - /** @return the last enumerated snapshot id */ + /** + * @return the last enumerated snapshot id + */ private CycleResult verifyOneCycle( ContinuousSplitPlannerImpl splitPlanner, IcebergEnumeratorPosition lastPosition) throws Exception { diff --git a/flink/v1.19/build.gradle b/flink/v1.19/build.gradle index 392a1cb124f0..50bcadb618e4 100644 --- a/flink/v1.19/build.gradle +++ b/flink/v1.19/build.gradle @@ -119,6 +119,7 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { testImplementation libs.awaitility testImplementation libs.assertj.core + testImplementation libs.sqlite.jdbc } test { @@ -127,7 +128,7 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { } project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") { - apply plugin: 'io.github.goooler.shadow' + apply plugin: 'com.gradleup.shadow' tasks.jar.dependsOn tasks.shadowJar diff --git a/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java b/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java index 007b423e592a..24cad2669dbe 100644 --- a/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java +++ b/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import java.nio.charset.StandardCharsets; import java.util.Comparator; import java.util.List; import java.util.Map; @@ -139,7 +140,7 @@ private static String randomString(String prefix) { buffer[i] = (byte) CHARS.charAt(ThreadLocalRandom.current().nextInt(CHARS.length())); } - return prefix + new String(buffer); + return prefix + new String(buffer, StandardCharsets.US_ASCII); } /** find the index where weightsUDF[index] < weight && weightsUDF[index+1] >= weight */ diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java index 7167859e600c..d5eea6706b39 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java @@ -53,6 +53,10 @@ public LongConfParser longConf() { return new LongConfParser(); } + public DoubleConfParser doubleConf() { + return new DoubleConfParser(); + } + public > EnumConfParser enumConfParser(Class enumClass) { return new EnumConfParser<>(enumClass); } @@ -135,6 +139,29 @@ public Long parseOptional() { } } + class DoubleConfParser extends ConfParser { + private Double defaultValue; + + @Override + protected DoubleConfParser self() { + return this; + } + + public DoubleConfParser defaultValue(double value) { + this.defaultValue = value; + return self(); + } + + public double parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(Double::parseDouble, defaultValue); + } + + public Double parseOptional() { + return parse(Double::parseDouble, null); + } + } + class StringConfParser extends ConfParser { private String defaultValue; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java index 4790dc85bf28..afc09a719f77 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java @@ -58,7 +58,9 @@ public class FlinkSchemaUtil { private FlinkSchemaUtil() {} - /** @deprecated Use {@link #convert(ResolvedSchema)} instead. */ + /** + * @deprecated Use {@link #convert(ResolvedSchema)} instead. + */ @Deprecated public static Schema convert(TableSchema schema) { LogicalType schemaType = schema.toRowDataType().getLogicalType(); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java index ca7b1120bc81..a31902d49a8b 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java @@ -26,6 +26,7 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.sink.shuffle.StatisticsType; /** * A class for common Iceberg configs for Flink writes. @@ -167,6 +168,26 @@ public DistributionMode distributionMode() { return DistributionMode.fromName(modeName); } + public StatisticsType rangeDistributionStatisticsType() { + String name = + confParser + .stringConf() + .option(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE.key()) + .flinkConfig(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE) + .defaultValue(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE.defaultValue()) + .parse(); + return StatisticsType.valueOf(name); + } + + public double rangeDistributionSortKeyBaseWeight() { + return confParser + .doubleConf() + .option(FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT.key()) + .flinkConfig(FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT) + .defaultValue(FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT.defaultValue()) + .parse(); + } + public int workerPoolSize() { return confParser .intConf() diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java index df73f2e09cac..c35286774874 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java @@ -23,6 +23,7 @@ import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.flink.sink.shuffle.StatisticsType; /** Flink sink write options */ public class FlinkWriteOptions { @@ -60,6 +61,19 @@ private FlinkWriteOptions() {} public static final ConfigOption DISTRIBUTION_MODE = ConfigOptions.key("distribution-mode").stringType().noDefaultValue(); + public static final ConfigOption RANGE_DISTRIBUTION_STATISTICS_TYPE = + ConfigOptions.key("range-distribution-statistics-type") + .stringType() + .defaultValue(StatisticsType.Auto.name()) + .withDescription("Type of statistics collection: Auto, Map, Sketch"); + + public static final ConfigOption RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT = + ConfigOptions.key("range-distribution-sort-key-base-weight") + .doubleType() + .defaultValue(0.0d) + .withDescription( + "Base weight for every sort key relative to target weight per writer task"); + // Branch to write to public static final ConfigOption BRANCH = ConfigOptions.key("branch").stringType().defaultValue(SnapshotRef.MAIN_BRANCH); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java index 1019285018d0..34576a1e5c0b 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java @@ -237,10 +237,10 @@ public MapData getMap(int pos) { @Override public RowData getRow(int pos, int numFields) { - return isNullAt(pos) ? null : getStructRowData(pos, numFields); + return isNullAt(pos) ? null : getStructRowData(pos); } - private StructRowData getStructRowData(int pos, int numFields) { + private StructRowData getStructRowData(int pos) { return new StructRowData( type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class)); } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/JdbcLockFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/JdbcLockFactory.java new file mode 100644 index 000000000000..085fbfecd270 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/JdbcLockFactory.java @@ -0,0 +1,323 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import java.io.IOException; +import java.sql.DatabaseMetaData; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.SQLNonTransientConnectionException; +import java.sql.SQLTimeoutException; +import java.sql.SQLTransientConnectionException; +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.jdbc.JdbcClientPool; +import org.apache.iceberg.jdbc.UncheckedInterruptedException; +import org.apache.iceberg.jdbc.UncheckedSQLException; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.PropertyUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * JDBC table backed implementation of the {@link + * org.apache.iceberg.flink.maintenance.operator.TriggerLockFactory}. + */ +public class JdbcLockFactory implements TriggerLockFactory { + private static final Logger LOG = LoggerFactory.getLogger(JdbcLockFactory.class); + + @VisibleForTesting + static final String INIT_LOCK_TABLES_PROPERTY = "flink-maintenance.lock.jdbc.init-lock-tables"; + + private static final String LOCK_TABLE_NAME = "flink_maintenance_lock"; + private static final int LOCK_ID_MAX_LENGTH = 100; + private static final String CREATE_LOCK_TABLE_SQL = + String.format( + "CREATE TABLE %s " + + "(LOCK_TYPE CHAR(1) NOT NULL, " + + "LOCK_ID VARCHAR(%s) NOT NULL, " + + "INSTANCE_ID CHAR(36) NOT NULL, PRIMARY KEY (LOCK_TYPE, LOCK_ID))", + LOCK_TABLE_NAME, LOCK_ID_MAX_LENGTH); + + private static final String CREATE_LOCK_SQL = + String.format( + "INSERT INTO %s (LOCK_TYPE, LOCK_ID, INSTANCE_ID) VALUES (?, ?, ?)", LOCK_TABLE_NAME); + private static final String GET_LOCK_SQL = + String.format("SELECT INSTANCE_ID FROM %s WHERE LOCK_TYPE=? AND LOCK_ID=?", LOCK_TABLE_NAME); + private static final String DELETE_LOCK_SQL = + String.format( + "DELETE FROM %s WHERE LOCK_TYPE=? AND LOCK_ID=? AND INSTANCE_ID=?", LOCK_TABLE_NAME); + + private final String uri; + private final String lockId; + private final Map properties; + private transient JdbcClientPool pool; + + /** + * Creates a new {@link TriggerLockFactory}. The lockId should be unique between the users of the + * same uri. + * + * @param uri of the jdbc connection + * @param lockId which should indentify the job and the table + * @param properties used for creating the jdbc connection pool + */ + public JdbcLockFactory(String uri, String lockId, Map properties) { + Preconditions.checkNotNull(uri, "JDBC connection URI is required"); + Preconditions.checkNotNull(properties, "Properties map is required"); + Preconditions.checkArgument( + lockId.length() < LOCK_ID_MAX_LENGTH, + "Invalid prefix length: lockId should be shorter than %s", + LOCK_ID_MAX_LENGTH); + this.uri = uri; + this.lockId = lockId; + this.properties = properties; + } + + @Override + public void open() { + this.pool = new JdbcClientPool(1, uri, properties); + + if (PropertyUtil.propertyAsBoolean(properties, INIT_LOCK_TABLES_PROPERTY, false)) { + initializeLockTables(); + } + } + + /** Only used in testing to share the jdbc pool */ + @VisibleForTesting + void open(JdbcLockFactory other) { + this.pool = other.pool; + } + + @Override + public Lock createLock() { + return new JdbcLock(pool, lockId, Type.MAINTENANCE); + } + + @Override + public Lock createRecoveryLock() { + return new JdbcLock(pool, lockId, Type.RECOVERY); + } + + @Override + public void close() throws IOException { + pool.close(); + } + + private void initializeLockTables() { + LOG.debug("Creating database tables (if missing) to store table maintenance locks"); + try { + pool.run( + conn -> { + DatabaseMetaData dbMeta = conn.getMetaData(); + ResultSet tableExists = + dbMeta.getTables( + null /* catalog name */, + null /* schemaPattern */, + LOCK_TABLE_NAME /* tableNamePattern */, + null /* types */); + if (tableExists.next()) { + LOG.debug("Flink maintenance lock table already exists"); + return true; + } + + LOG.info("Creating Flink maintenance lock table {}", LOCK_TABLE_NAME); + return conn.prepareStatement(CREATE_LOCK_TABLE_SQL).execute(); + }); + + } catch (SQLTimeoutException e) { + throw new UncheckedSQLException( + e, "Cannot initialize JDBC table maintenance lock: Query timed out"); + } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) { + throw new UncheckedSQLException( + e, "Cannot initialize JDBC table maintenance lock: Connection failed"); + } catch (SQLException e) { + throw new UncheckedSQLException(e, "Cannot initialize JDBC table maintenance lock"); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted in call to initialize"); + } + } + + private static class JdbcLock implements TriggerLockFactory.Lock { + private final JdbcClientPool pool; + private final String lockId; + private final Type type; + + private JdbcLock(JdbcClientPool pool, String lockId, Type type) { + this.pool = pool; + this.lockId = lockId; + this.type = type; + } + + @Override + public boolean tryLock() { + if (isHeld()) { + LOG.info("Lock is already held for {}", this); + return false; + } + + String newInstanceId = UUID.randomUUID().toString(); + try { + return pool.run( + conn -> { + try (PreparedStatement sql = conn.prepareStatement(CREATE_LOCK_SQL)) { + sql.setString(1, type.key); + sql.setString(2, lockId); + sql.setString(3, newInstanceId); + int count = sql.executeUpdate(); + LOG.info( + "Created {} lock with instanceId {} with row count {}", + this, + newInstanceId, + count); + return count == 1; + } + }); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted during tryLock"); + } catch (SQLException e) { + // SQL exception happened when creating the lock. Check if the lock creation was + // successful behind the scenes. + if (newInstanceId.equals(instanceId())) { + return true; + } else { + throw new UncheckedSQLException(e, "Failed to create %s lock", this); + } + } + } + + @SuppressWarnings("checkstyle:NestedTryDepth") + @Override + public boolean isHeld() { + try { + return pool.run( + conn -> { + try (PreparedStatement sql = conn.prepareStatement(GET_LOCK_SQL)) { + sql.setString(1, type.key); + sql.setString(2, lockId); + try (ResultSet rs = sql.executeQuery()) { + return rs.next(); + } + } + }); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted during isHeld"); + } catch (SQLException e) { + // SQL exception happened when getting lock information + throw new UncheckedSQLException(e, "Failed to check the state of the lock %s", this); + } + } + + @SuppressWarnings("checkstyle:NestedTryDepth") + @Override + public void unlock() { + try { + // Possible concurrency issue: + // - `unlock` and `tryLock` happens at the same time when there is an existing lock + // + // Steps: + // 1. `unlock` removes the lock in the database, but there is a temporary connection failure + // 2. `lock` founds that there is no lock, so creates a new lock + // 3. `unlock` retires the lock removal and removes the new lock + // + // To prevent the situation above we fetch the current lockId, and remove the lock + // only with the given id. + String instanceId = instanceId(); + + if (instanceId != null) { + pool.run( + conn -> { + try (PreparedStatement sql = conn.prepareStatement(DELETE_LOCK_SQL)) { + sql.setString(1, type.key); + sql.setString(2, lockId); + sql.setString(3, instanceId); + long count = sql.executeUpdate(); + LOG.info( + "Deleted {} lock with instanceId {} with row count {}", + this, + instanceId, + count); + } catch (SQLException e) { + // SQL exception happened when deleting lock information + throw new UncheckedSQLException( + e, "Failed to delete %s lock with instanceId %s", this, instanceId); + } + + return null; + }); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted during unlock"); + } catch (SQLException e) { + // SQL exception happened when getting/updating lock information + throw new UncheckedSQLException(e, "Failed to remove lock %s", this); + } + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this).add("type", type).add("lockId", lockId).toString(); + } + + @SuppressWarnings("checkstyle:NestedTryDepth") + private String instanceId() { + try { + return pool.run( + conn -> { + try (PreparedStatement sql = conn.prepareStatement(GET_LOCK_SQL)) { + sql.setString(1, type.key); + sql.setString(2, lockId); + try (ResultSet rs = sql.executeQuery()) { + if (rs.next()) { + return rs.getString(1); + } else { + return null; + } + } + } catch (SQLException e) { + // SQL exception happened when getting lock information + throw new UncheckedSQLException(e, "Failed to get lock information for %s", type); + } + }); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted during unlock"); + } catch (SQLException e) { + throw new UncheckedSQLException(e, "Failed to get lock information for %s", type); + } + } + } + + private enum Type { + MAINTENANCE("m"), + RECOVERY("r"); + + private final String key; + + Type(String key) { + this.key = key; + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java new file mode 100644 index 000000000000..3c3761ef2f4d --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java @@ -0,0 +1,144 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import java.util.List; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.Counter; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Manages locks and collect {@link org.apache.flink.metrics.Metric} for the Maintenance Tasks. + * + *

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

    + *
  • Every {@link TaskResult} is followed by a {@link Watermark} for normal {@link Trigger}s + *
  • For the {@link Trigger#recovery(long)} {@link Watermark} there is no element to process + *
+ * + * When processing the inputs there are 3 possibilities: + * + *
    + *
  • Normal execution - we receive a {@link TaskResult} and then a {@link Watermark} - unlocking + * the lock is handled by the {@link #processElement(StreamRecord)} + *
  • Recovery without ongoing execution (unlocking the recoveryLock) - we receive the {@link + * Trigger#recovery(long)} {@link Watermark} without any {@link TaskResult} - unlocking the + * {@link TriggerLockFactory#createRecoveryLock()} and a possible {@link + * TriggerLockFactory#createLock()} is handled by the {@link #processWatermark(Watermark)} + * (the {@link #lastProcessedTaskStartEpoch} is 0 in this case) + *
  • Recovery with an ongoing execution - we receive a {@link TaskResult} and then a {@link + * Watermark} - unlocking the {@link TriggerLockFactory#createLock()} is handled by the {@link + * #processElement(StreamRecord)}, unlocking the {@link + * TriggerLockFactory#createRecoveryLock()} is handled by the {@link + * #processWatermark(Watermark)} (the {@link #lastProcessedTaskStartEpoch} is the start time + * of the old task) + *
+ */ +@Internal +public class LockRemover extends AbstractStreamOperator + implements OneInputStreamOperator { + private static final Logger LOG = LoggerFactory.getLogger(LockRemover.class); + + private final TriggerLockFactory lockFactory; + private final List maintenanceTaskNames; + + private transient List succeededTaskResultCounters; + private transient List failedTaskResultCounters; + private transient List taskLastRunDurationMs; + private transient TriggerLockFactory.Lock lock; + private transient TriggerLockFactory.Lock recoveryLock; + private transient long lastProcessedTaskStartEpoch = 0L; + + public LockRemover(TriggerLockFactory lockFactory, List maintenanceTaskNames) { + Preconditions.checkNotNull(lockFactory, "Lock factory should no be null"); + Preconditions.checkArgument( + maintenanceTaskNames != null && !maintenanceTaskNames.isEmpty(), + "Invalid maintenance task names: null or empty"); + + this.lockFactory = lockFactory; + this.maintenanceTaskNames = maintenanceTaskNames; + } + + @Override + public void open() throws Exception { + super.open(); + this.succeededTaskResultCounters = + Lists.newArrayListWithExpectedSize(maintenanceTaskNames.size()); + this.failedTaskResultCounters = Lists.newArrayListWithExpectedSize(maintenanceTaskNames.size()); + this.taskLastRunDurationMs = Lists.newArrayListWithExpectedSize(maintenanceTaskNames.size()); + for (String name : maintenanceTaskNames) { + succeededTaskResultCounters.add( + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.SUCCEEDED_TASK_COUNTER)); + failedTaskResultCounters.add( + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.FAILED_TASK_COUNTER)); + AtomicLong duration = new AtomicLong(0); + taskLastRunDurationMs.add(duration); + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .gauge(TableMaintenanceMetrics.LAST_RUN_DURATION_MS, duration::get); + } + + this.lock = lockFactory.createLock(); + this.recoveryLock = lockFactory.createRecoveryLock(); + } + + @Override + public void processElement(StreamRecord streamRecord) { + TaskResult taskResult = streamRecord.getValue(); + LOG.info( + "Processing result {} for task {}", + taskResult, + maintenanceTaskNames.get(taskResult.taskIndex())); + long duration = System.currentTimeMillis() - taskResult.startEpoch(); + lock.unlock(); + this.lastProcessedTaskStartEpoch = taskResult.startEpoch(); + + // Update the metrics + taskLastRunDurationMs.get(taskResult.taskIndex()).set(duration); + if (taskResult.success()) { + succeededTaskResultCounters.get(taskResult.taskIndex()).inc(); + } else { + failedTaskResultCounters.get(taskResult.taskIndex()).inc(); + } + } + + @Override + public void processWatermark(Watermark mark) { + if (mark.getTimestamp() > lastProcessedTaskStartEpoch) { + lock.unlock(); + recoveryLock.unlock(); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java index d74b2349b1de..89efffa15f16 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java @@ -43,7 +43,7 @@ /** Monitors an Iceberg table for changes */ @Internal -public class MonitorSource extends SingleThreadedIteratorSource { +class MonitorSource extends SingleThreadedIteratorSource { private static final Logger LOG = LoggerFactory.getLogger(MonitorSource.class); private final TableLoader tableLoader; @@ -58,7 +58,7 @@ public class MonitorSource extends SingleThreadedIteratorSource { * @param rateLimiterStrategy limits the frequency the table is checked * @param maxReadBack sets the number of snapshots read before stopping change collection */ - public MonitorSource( + MonitorSource( TableLoader tableLoader, RateLimiterStrategy rateLimiterStrategy, long maxReadBack) { Preconditions.checkNotNull(tableLoader, "Table loader should no be null"); Preconditions.checkNotNull(rateLimiterStrategy, "Rate limiter strategy should no be null"); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java index 452ed80ed0e5..773b34b6c495 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java @@ -29,19 +29,29 @@ /** Event describing changes in an Iceberg table */ @Internal class TableChange { - private int dataFileNum; - private int deleteFileNum; - private long dataFileSize; - private long deleteFileSize; - private int commitNum; + private int dataFileCount; + private long dataFileSizeInBytes; + private int posDeleteFileCount; + private long posDeleteRecordCount; + private int eqDeleteFileCount; + private long eqDeleteRecordCount; + private int commitCount; TableChange( - int dataFileNum, int deleteFileNum, long dataFileSize, long deleteFileSize, int commitNum) { - this.dataFileNum = dataFileNum; - this.deleteFileNum = deleteFileNum; - this.dataFileSize = dataFileSize; - this.deleteFileSize = deleteFileSize; - this.commitNum = commitNum; + int dataFileCount, + long dataFileSizeInBytes, + int posDeleteFileCount, + long posDeleteRecordCount, + int eqDeleteFileCount, + long eqDeleteRecordCount, + int commitCount) { + this.dataFileCount = dataFileCount; + this.dataFileSizeInBytes = dataFileSizeInBytes; + this.posDeleteFileCount = posDeleteFileCount; + this.posDeleteRecordCount = posDeleteRecordCount; + this.eqDeleteFileCount = eqDeleteFileCount; + this.eqDeleteRecordCount = eqDeleteRecordCount; + this.commitCount = commitCount; } TableChange(Snapshot snapshot, FileIO io) { @@ -50,63 +60,96 @@ class TableChange { dataFiles.forEach( dataFile -> { - this.dataFileNum++; - this.dataFileSize += dataFile.fileSizeInBytes(); + this.dataFileCount++; + this.dataFileSizeInBytes += dataFile.fileSizeInBytes(); }); deleteFiles.forEach( deleteFile -> { - this.deleteFileNum++; - this.deleteFileSize += deleteFile.fileSizeInBytes(); + switch (deleteFile.content()) { + case POSITION_DELETES: + this.posDeleteFileCount++; + this.posDeleteRecordCount += deleteFile.recordCount(); + break; + case EQUALITY_DELETES: + this.eqDeleteFileCount++; + this.eqDeleteRecordCount += deleteFile.recordCount(); + break; + default: + throw new IllegalArgumentException("Unexpected delete file content: " + deleteFile); + } }); - this.commitNum = 1; + this.commitCount = 1; } static TableChange empty() { - return new TableChange(0, 0, 0L, 0L, 0); + return new TableChange(0, 0L, 0, 0L, 0, 0L, 0); } - int dataFileNum() { - return dataFileNum; + static Builder builder() { + return new Builder(); } - int deleteFileNum() { - return deleteFileNum; + int dataFileCount() { + return dataFileCount; } - long dataFileSize() { - return dataFileSize; + long dataFileSizeInBytes() { + return dataFileSizeInBytes; } - long deleteFileSize() { - return deleteFileSize; + int posDeleteFileCount() { + return posDeleteFileCount; } - public int commitNum() { - return commitNum; + long posDeleteRecordCount() { + return posDeleteRecordCount; + } + + int eqDeleteFileCount() { + return eqDeleteFileCount; + } + + long eqDeleteRecordCount() { + return eqDeleteRecordCount; + } + + public int commitCount() { + return commitCount; } public void merge(TableChange other) { - this.dataFileNum += other.dataFileNum; - this.deleteFileNum += other.deleteFileNum; - this.dataFileSize += other.dataFileSize; - this.deleteFileSize += other.deleteFileSize; - this.commitNum += other.commitNum; + this.dataFileCount += other.dataFileCount; + this.dataFileSizeInBytes += other.dataFileSizeInBytes; + this.posDeleteFileCount += other.posDeleteFileCount; + this.posDeleteRecordCount += other.posDeleteRecordCount; + this.eqDeleteFileCount += other.eqDeleteFileCount; + this.eqDeleteRecordCount += other.eqDeleteRecordCount; + this.commitCount += other.commitCount; } TableChange copy() { - return new TableChange(dataFileNum, deleteFileNum, dataFileSize, deleteFileSize, commitNum); + return new TableChange( + dataFileCount, + dataFileSizeInBytes, + posDeleteFileCount, + posDeleteRecordCount, + eqDeleteFileCount, + eqDeleteRecordCount, + commitCount); } @Override public String toString() { return MoreObjects.toStringHelper(this) - .add("dataFileNum", dataFileNum) - .add("deleteFileNum", deleteFileNum) - .add("dataFileSize", dataFileSize) - .add("deleteFileSize", deleteFileSize) - .add("commitNum", commitNum) + .add("dataFileCount", dataFileCount) + .add("dataFileSizeInBytes", dataFileSizeInBytes) + .add("posDeleteFileCount", posDeleteFileCount) + .add("posDeleteRecordCount", posDeleteRecordCount) + .add("eqDeleteFileCount", eqDeleteFileCount) + .add("eqDeleteRecordCount", eqDeleteRecordCount) + .add("commitCount", commitCount) .toString(); } @@ -119,15 +162,82 @@ public boolean equals(Object other) { } TableChange that = (TableChange) other; - return dataFileNum == that.dataFileNum - && deleteFileNum == that.deleteFileNum - && dataFileSize == that.dataFileSize - && deleteFileSize == that.deleteFileSize - && commitNum == that.commitNum; + return dataFileCount == that.dataFileCount + && dataFileSizeInBytes == that.dataFileSizeInBytes + && posDeleteFileCount == that.posDeleteFileCount + && posDeleteRecordCount == that.posDeleteRecordCount + && eqDeleteFileCount == that.eqDeleteFileCount + && eqDeleteRecordCount == that.eqDeleteRecordCount + && commitCount == that.commitCount; } @Override public int hashCode() { - return Objects.hash(dataFileNum, deleteFileNum, dataFileSize, deleteFileSize, commitNum); + return Objects.hash( + dataFileCount, + dataFileSizeInBytes, + posDeleteFileCount, + posDeleteRecordCount, + eqDeleteFileCount, + eqDeleteRecordCount, + commitCount); + } + + static class Builder { + private int dataFileCount = 0; + private long dataFileSizeInBytes = 0L; + private int posDeleteFileCount = 0; + private long posDeleteRecordCount = 0L; + private int eqDeleteFileCount = 0; + private long eqDeleteRecordCount = 0L; + private int commitCount = 0; + + private Builder() {} + + public Builder dataFileCount(int newDataFileCount) { + this.dataFileCount = newDataFileCount; + return this; + } + + public Builder dataFileSizeInBytes(long newDataFileSizeInBytes) { + this.dataFileSizeInBytes = newDataFileSizeInBytes; + return this; + } + + public Builder posDeleteFileCount(int newPosDeleteFileCount) { + this.posDeleteFileCount = newPosDeleteFileCount; + return this; + } + + public Builder posDeleteRecordCount(long newPosDeleteRecordCount) { + this.posDeleteRecordCount = newPosDeleteRecordCount; + return this; + } + + public Builder eqDeleteFileCount(int newEqDeleteFileCount) { + this.eqDeleteFileCount = newEqDeleteFileCount; + return this; + } + + public Builder eqDeleteRecordCount(long newEqDeleteRecordCount) { + this.eqDeleteRecordCount = newEqDeleteRecordCount; + return this; + } + + public Builder commitCount(int newCommitCount) { + this.commitCount = newCommitCount; + return this; + } + + public TableChange build() { + return new TableChange( + dataFileCount, + dataFileSizeInBytes, + posDeleteFileCount, + posDeleteRecordCount, + eqDeleteFileCount, + eqDeleteRecordCount, + commitCount); + } } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java new file mode 100644 index 000000000000..1a04461aed43 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +public class TableMaintenanceMetrics { + public static final String GROUP_KEY = "maintenanceTask"; + public static final String GROUP_VALUE_DEFAULT = "maintenanceTask"; + + // TriggerManager metrics + public static final String RATE_LIMITER_TRIGGERED = "rateLimiterTriggered"; + public static final String CONCURRENT_RUN_THROTTLED = "concurrentRunThrottled"; + public static final String TRIGGERED = "triggered"; + public static final String NOTHING_TO_TRIGGER = "nothingToTrigger"; + + // LockRemover metrics + public static final String SUCCEEDED_TASK_COUNTER = "succeededTasks"; + public static final String FAILED_TASK_COUNTER = "failedTasks"; + public static final String LAST_RUN_DURATION_MS = "lastRunDurationMs"; + + private TableMaintenanceMetrics() { + // do not instantiate + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TaskResult.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TaskResult.java new file mode 100644 index 000000000000..06f10f1c1d68 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TaskResult.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import java.util.List; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +/** The result of a single Maintenance Task. */ +@Internal +public class TaskResult { + private final int taskIndex; + private final long startEpoch; + private final boolean success; + private final List exceptions; + + public TaskResult(int taskIndex, long startEpoch, boolean success, List exceptions) { + this.taskIndex = taskIndex; + this.startEpoch = startEpoch; + this.success = success; + this.exceptions = exceptions; + } + + public int taskIndex() { + return taskIndex; + } + + public long startEpoch() { + return startEpoch; + } + + public boolean success() { + return success; + } + + public List exceptions() { + return exceptions; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("taskIndex", taskIndex) + .add("startEpoch", startEpoch) + .add("success", success) + .add("exceptions", exceptions) + .toString(); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/Trigger.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/Trigger.java new file mode 100644 index 000000000000..85c6c8dbdd55 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/Trigger.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +@Internal +class Trigger { + private final long timestamp; + private final SerializableTable table; + private final Integer taskId; + private final boolean isRecovery; + + private Trigger(long timestamp, SerializableTable table, Integer taskId, boolean isRecovery) { + this.timestamp = timestamp; + this.table = table; + this.taskId = taskId; + this.isRecovery = isRecovery; + } + + static Trigger create(long timestamp, SerializableTable table, int taskId) { + return new Trigger(timestamp, table, taskId, false); + } + + static Trigger recovery(long timestamp) { + return new Trigger(timestamp, null, null, true); + } + + long timestamp() { + return timestamp; + } + + SerializableTable table() { + return table; + } + + Integer taskId() { + return taskId; + } + + boolean isRecovery() { + return isRecovery; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("timestamp", timestamp) + .add("table", table == null ? null : table.name()) + .add("taskId", taskId) + .add("isRecovery", isRecovery) + .toString(); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java new file mode 100644 index 000000000000..dba33b22a42a --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import java.io.Serializable; +import java.time.Duration; +import java.util.List; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Internal +class TriggerEvaluator implements Serializable { + private static final Logger LOG = LoggerFactory.getLogger(TriggerEvaluator.class); + private final List predicates; + + private TriggerEvaluator(List predicates) { + Preconditions.checkArgument(!predicates.isEmpty(), "Provide at least 1 condition."); + + this.predicates = predicates; + } + + boolean check(TableChange event, long lastTimeMs, long currentTimeMs) { + boolean result = + predicates.stream().anyMatch(p -> p.evaluate(event, lastTimeMs, currentTimeMs)); + LOG.debug( + "Checking event: {}, at {}, last: {} with result: {}", + event, + currentTimeMs, + lastTimeMs, + result); + return result; + } + + static class Builder implements Serializable { + private Integer dataFileCount; + private Long dataFileSizeInBytes; + private Integer posDeleteFileCount; + private Long posDeleteRecordCount; + private Integer eqDeleteFileCount; + private Long eqDeleteRecordCount; + private Integer commitCount; + private Duration timeout; + + public Builder dataFileCount(int newDataFileCount) { + this.dataFileCount = newDataFileCount; + return this; + } + + public Builder dataFileSizeInBytes(long neDataFileSizeInBytes) { + this.dataFileSizeInBytes = neDataFileSizeInBytes; + return this; + } + + public Builder posDeleteFileCount(int newPosDeleteFileCount) { + this.posDeleteFileCount = newPosDeleteFileCount; + return this; + } + + public Builder posDeleteRecordCount(long newPosDeleteRecordCount) { + this.posDeleteRecordCount = newPosDeleteRecordCount; + return this; + } + + public Builder eqDeleteFileCount(int newEqDeleteFileCount) { + this.eqDeleteFileCount = newEqDeleteFileCount; + return this; + } + + public Builder eqDeleteRecordCount(long newEqDeleteRecordCount) { + this.eqDeleteRecordCount = newEqDeleteRecordCount; + return this; + } + + public Builder commitCount(int newCommitCount) { + this.commitCount = newCommitCount; + return this; + } + + Builder timeout(Duration newTimeout) { + this.timeout = newTimeout; + return this; + } + + TriggerEvaluator build() { + List predicates = Lists.newArrayList(); + if (dataFileCount != null) { + predicates.add((change, unused, unused2) -> change.dataFileCount() >= dataFileCount); + } + + if (dataFileSizeInBytes != null) { + predicates.add( + (change, unused, unused2) -> change.dataFileSizeInBytes() >= dataFileSizeInBytes); + } + + if (posDeleteFileCount != null) { + predicates.add( + (change, unused, unused2) -> change.posDeleteFileCount() >= posDeleteFileCount); + } + + if (posDeleteRecordCount != null) { + predicates.add( + (change, unused, unused2) -> change.posDeleteRecordCount() >= posDeleteRecordCount); + } + + if (eqDeleteFileCount != null) { + predicates.add( + (change, unused, unused2) -> change.eqDeleteFileCount() >= eqDeleteFileCount); + } + + if (eqDeleteRecordCount != null) { + predicates.add( + (change, unused, unused2) -> change.eqDeleteRecordCount() >= eqDeleteRecordCount); + } + + if (commitCount != null) { + predicates.add((change, unused, unused2) -> change.commitCount() >= commitCount); + } + + if (timeout != null) { + predicates.add( + (change, lastTimeMs, currentTimeMs) -> + currentTimeMs - lastTimeMs >= timeout.toMillis()); + } + + return new TriggerEvaluator(predicates); + } + } + + private interface Predicate extends Serializable { + boolean evaluate(TableChange event, long lastTimeMs, long currentTimeMs); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerLockFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerLockFactory.java new file mode 100644 index 000000000000..329223d27ccf --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerLockFactory.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import java.io.Closeable; +import java.io.Serializable; +import org.apache.flink.annotation.Experimental; + +/** Lock interface for handling locks for the Flink Table Maintenance jobs. */ +@Experimental +public interface TriggerLockFactory extends Serializable, Closeable { + void open(); + + Lock createLock(); + + Lock createRecoveryLock(); + + interface Lock { + /** + * Tries to acquire a lock with a given key. Anyone already holding a lock would prevent + * acquiring this lock. Not reentrant. + * + *

Called by {@link TriggerManager}. Implementations could assume that are no concurrent + * calls for this method. + * + * @return true if the lock is acquired by this job, false if the lock + * is already held by someone + */ + boolean tryLock(); + + /** + * Checks if the lock is already taken. + * + * @return true if the lock is held by someone + */ + boolean isHeld(); + + // TODO: Fix the link to the LockRemover when we have a final name and implementation + /** + * Releases the lock. Should not fail if the lock is not held by anyone. + * + *

Called by LockRemover. Implementations could assume that are no concurrent calls for this + * method. + */ + void unlock(); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java new file mode 100644 index 000000000000..dc95b27af0a6 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java @@ -0,0 +1,341 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.streaming.api.TimerService; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.functions.KeyedProcessFunction; +import org.apache.flink.util.Collector; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * TriggerManager starts the Maintenance Tasks by emitting {@link Trigger} messages which are + * calculated based on the incoming {@link TableChange} messages. The TriggerManager keeps track of + * the changes since the last run of the Maintenance Tasks and triggers a new run based on the + * result of the {@link TriggerEvaluator}. + * + *

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

The TriggerManager should run as a global operator. {@link KeyedProcessFunction} is used, so + * the timer functions are available, but the key is not used. + */ +@Internal +class TriggerManager extends KeyedProcessFunction + implements CheckpointedFunction { + private static final Logger LOG = LoggerFactory.getLogger(TriggerManager.class); + + private final TableLoader tableLoader; + private final TriggerLockFactory lockFactory; + private final List maintenanceTaskNames; + private final List evaluators; + private final long minFireDelayMs; + private final long lockCheckDelayMs; + private transient Counter rateLimiterTriggeredCounter; + private transient Counter concurrentRunThrottledCounter; + private transient Counter nothingToTriggerCounter; + private transient List triggerCounters; + private transient ValueState nextEvaluationTimeState; + private transient ListState accumulatedChangesState; + private transient ListState lastTriggerTimesState; + private transient Long nextEvaluationTime; + private transient List accumulatedChanges; + private transient List lastTriggerTimes; + private transient TriggerLockFactory.Lock lock; + private transient TriggerLockFactory.Lock recoveryLock; + private transient boolean shouldRestoreTasks = false; + private transient boolean inited = false; + // To keep the task scheduling fair we keep the last triggered task position in memory. + // If we find a task to trigger, then we run it, but after it is finished, we start from the given + // position to prevent "starvation" of the tasks. + // When there is nothing to trigger, we start from the beginning, as the order of the tasks might + // be important (RewriteDataFiles first, and then RewriteManifestFiles later) + private transient int startsFrom = 0; + private transient boolean triggered = false; + + TriggerManager( + TableLoader tableLoader, + TriggerLockFactory lockFactory, + List maintenanceTaskNames, + List evaluators, + long minFireDelayMs, + long lockCheckDelayMs) { + Preconditions.checkNotNull(tableLoader, "Table loader should no be null"); + Preconditions.checkNotNull(lockFactory, "Lock factory should no be null"); + Preconditions.checkArgument( + maintenanceTaskNames != null && !maintenanceTaskNames.isEmpty(), + "Invalid maintenance task names: null or empty"); + Preconditions.checkArgument( + evaluators != null && !evaluators.isEmpty(), "Invalid evaluators: null or empty"); + Preconditions.checkArgument( + maintenanceTaskNames.size() == evaluators.size(), + "Provide a name and evaluator for all of the maintenance tasks"); + Preconditions.checkArgument(minFireDelayMs > 0, "Minimum fire delay should be at least 1."); + Preconditions.checkArgument( + lockCheckDelayMs > 0, "Minimum lock delay rate should be at least 1 ms."); + + this.tableLoader = tableLoader; + this.lockFactory = lockFactory; + this.maintenanceTaskNames = maintenanceTaskNames; + this.evaluators = evaluators; + this.minFireDelayMs = minFireDelayMs; + this.lockCheckDelayMs = lockCheckDelayMs; + } + + @Override + public void open(Configuration parameters) throws Exception { + this.rateLimiterTriggeredCounter = + getRuntimeContext() + .getMetricGroup() + .addGroup( + TableMaintenanceMetrics.GROUP_KEY, TableMaintenanceMetrics.GROUP_VALUE_DEFAULT) + .counter(TableMaintenanceMetrics.RATE_LIMITER_TRIGGERED); + this.concurrentRunThrottledCounter = + getRuntimeContext() + .getMetricGroup() + .addGroup( + TableMaintenanceMetrics.GROUP_KEY, TableMaintenanceMetrics.GROUP_VALUE_DEFAULT) + .counter(TableMaintenanceMetrics.CONCURRENT_RUN_THROTTLED); + this.nothingToTriggerCounter = + getRuntimeContext() + .getMetricGroup() + .addGroup( + TableMaintenanceMetrics.GROUP_KEY, TableMaintenanceMetrics.GROUP_VALUE_DEFAULT) + .counter(TableMaintenanceMetrics.NOTHING_TO_TRIGGER); + this.triggerCounters = + maintenanceTaskNames.stream() + .map( + name -> + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.TRIGGERED)) + .collect(Collectors.toList()); + + this.nextEvaluationTimeState = + getRuntimeContext() + .getState(new ValueStateDescriptor<>("triggerManagerNextTriggerTime", Types.LONG)); + this.accumulatedChangesState = + getRuntimeContext() + .getListState( + new ListStateDescriptor<>( + "triggerManagerAccumulatedChange", TypeInformation.of(TableChange.class))); + this.lastTriggerTimesState = + getRuntimeContext() + .getListState(new ListStateDescriptor<>("triggerManagerLastTriggerTime", Types.LONG)); + + tableLoader.open(); + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + if (inited) { + // Only store state if initialized + nextEvaluationTimeState.update(nextEvaluationTime); + accumulatedChangesState.update(accumulatedChanges); + lastTriggerTimesState.update(lastTriggerTimes); + LOG.info( + "Storing state: nextEvaluationTime {}, accumulatedChanges {}, lastTriggerTimes {}", + nextEvaluationTime, + accumulatedChanges, + lastTriggerTimes); + } else { + LOG.info("Not initialized, state is not stored"); + } + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + LOG.info("Initializing state restored: {}", context.isRestored()); + lockFactory.open(); + this.lock = lockFactory.createLock(); + this.recoveryLock = lockFactory.createRecoveryLock(); + if (context.isRestored()) { + shouldRestoreTasks = true; + } + } + + @Override + public void processElement(TableChange change, Context ctx, Collector out) + throws Exception { + init(out, ctx.timerService()); + + accumulatedChanges.forEach(tableChange -> tableChange.merge(change)); + + long current = ctx.timerService().currentProcessingTime(); + if (nextEvaluationTime == null) { + checkAndFire(current, ctx.timerService(), out); + } else { + LOG.info( + "Trigger manager rate limiter triggered current: {}, next: {}, accumulated changes: {}", + current, + nextEvaluationTime, + accumulatedChanges); + rateLimiterTriggeredCounter.inc(); + } + } + + @Override + public void onTimer(long timestamp, OnTimerContext ctx, Collector out) throws Exception { + init(out, ctx.timerService()); + this.nextEvaluationTime = null; + checkAndFire(ctx.timerService().currentProcessingTime(), ctx.timerService(), out); + } + + @Override + public void close() throws IOException { + tableLoader.close(); + lockFactory.close(); + } + + private void checkAndFire(long current, TimerService timerService, Collector out) { + if (shouldRestoreTasks) { + if (recoveryLock.isHeld()) { + // Recovered tasks in progress. Skip trigger check + LOG.debug("The recovery lock is still held at {}", current); + schedule(timerService, current + lockCheckDelayMs); + return; + } else { + LOG.info("The recovery is finished at {}", current); + shouldRestoreTasks = false; + } + } + + Integer taskToStart = + nextTrigger(evaluators, accumulatedChanges, lastTriggerTimes, current, startsFrom); + if (taskToStart == null) { + // Nothing to execute + if (!triggered) { + nothingToTriggerCounter.inc(); + LOG.debug("Nothing to execute at {} for collected: {}", current, accumulatedChanges); + } else { + LOG.debug("Execution check finished"); + } + + // Next time start from the beginning + startsFrom = 0; + triggered = false; + return; + } + + if (lock.tryLock()) { + TableChange change = accumulatedChanges.get(taskToStart); + SerializableTable table = + (SerializableTable) SerializableTable.copyOf(tableLoader.loadTable()); + out.collect(Trigger.create(current, table, taskToStart)); + LOG.debug("Fired event with time: {}, collected: {} for {}", current, change, table.name()); + triggerCounters.get(taskToStart).inc(); + accumulatedChanges.set(taskToStart, TableChange.empty()); + lastTriggerTimes.set(taskToStart, current); + schedule(timerService, current + minFireDelayMs); + startsFrom = (taskToStart + 1) % evaluators.size(); + triggered = true; + } else { + // A task is already running, waiting for it to finish + LOG.info("Failed to acquire lock. Delaying task to {}", current + lockCheckDelayMs); + + startsFrom = taskToStart; + concurrentRunThrottledCounter.inc(); + schedule(timerService, current + lockCheckDelayMs); + } + + timerService.registerProcessingTimeTimer(nextEvaluationTime); + } + + private void schedule(TimerService timerService, long time) { + this.nextEvaluationTime = time; + timerService.registerProcessingTimeTimer(time); + } + + private static Integer nextTrigger( + List evaluators, + List changes, + List lastTriggerTimes, + long currentTime, + int startPos) { + int current = startPos; + do { + if (evaluators + .get(current) + .check(changes.get(current), lastTriggerTimes.get(current), currentTime)) { + return current; + } + + current = (current + 1) % evaluators.size(); + } while (current != startPos); + + return null; + } + + private void init(Collector out, TimerService timerService) throws Exception { + if (!inited) { + long current = timerService.currentProcessingTime(); + + // Initialize from state + this.nextEvaluationTime = nextEvaluationTimeState.value(); + this.accumulatedChanges = Lists.newArrayList(accumulatedChangesState.get()); + this.lastTriggerTimes = Lists.newArrayList(lastTriggerTimesState.get()); + + // Initialize if the state was empty + if (accumulatedChanges.isEmpty()) { + for (int i = 0; i < evaluators.size(); ++i) { + accumulatedChanges.add(TableChange.empty()); + lastTriggerTimes.add(current); + } + } + + if (shouldRestoreTasks) { + // When the job state is restored, there could be ongoing tasks. + // To prevent collision with the new triggers the following is done: + // - add a recovery lock + // - fire a recovery trigger + // This ensures that the tasks of the previous trigger are executed, and the lock is removed + // in the end. The result of the 'tryLock' is ignored as an already existing lock prevents + // collisions as well. + recoveryLock.tryLock(); + out.collect(Trigger.recovery(current)); + if (nextEvaluationTime == null) { + schedule(timerService, current + minFireDelayMs); + } + } + + inited = true; + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java index c7e8a2dea7cb..9571efdc5268 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java @@ -33,9 +33,14 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; class FlinkManifestUtil { + + private static final Logger LOG = LoggerFactory.getLogger(FlinkManifestUtil.class); private static final int FORMAT_V2 = 2; private static final Long DUMMY_SNAPSHOT_ID = 0L; @@ -129,4 +134,26 @@ static WriteResult readCompletedFiles( return builder.addReferencedDataFiles(deltaManifests.referencedDataFiles()).build(); } + + static void deleteCommittedManifests( + Table table, List manifests, String newFlinkJobId, long checkpointId) { + for (ManifestFile manifest : manifests) { + try { + table.io().deleteFile(manifest.path()); + } catch (Exception e) { + // The flink manifests cleaning failure shouldn't abort the completed checkpoint. + String details = + MoreObjects.toStringHelper(FlinkManifestUtil.class) + .add("tableName", table.name()) + .add("flinkJobId", newFlinkJobId) + .add("checkpointId", checkpointId) + .add("manifestPath", manifest.path()) + .toString(); + LOG.warn( + "The iceberg transaction has been committed, but we failed to clean the temporary flink manifests: {}", + details, + e); + } + } + } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java index 769af7d77140..c53431490984 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java @@ -53,15 +53,20 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Partitioning; import org.apache.iceberg.Schema; import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.SortOrder; import org.apache.iceberg.Table; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.FlinkWriteConf; import org.apache.iceberg.flink.FlinkWriteOptions; import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.sink.shuffle.DataStatisticsOperatorFactory; +import org.apache.iceberg.flink.sink.shuffle.RangePartitioner; +import org.apache.iceberg.flink.sink.shuffle.StatisticsOrRecord; +import org.apache.iceberg.flink.sink.shuffle.StatisticsType; import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; -import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -233,15 +238,68 @@ public Builder flinkConf(ReadableConfig config) { * @return {@link Builder} to connect the iceberg table. */ public Builder distributionMode(DistributionMode mode) { - Preconditions.checkArgument( - !DistributionMode.RANGE.equals(mode), - "Flink does not support 'range' write distribution mode now."); if (mode != null) { writeOptions.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), mode.modeName()); } return this; } + /** + * Range distribution needs to collect statistics about data distribution to properly shuffle + * the records in relatively balanced way. In general, low cardinality should use {@link + * StatisticsType#Map} and high cardinality should use {@link StatisticsType#Sketch} Refer to + * {@link StatisticsType} Javadoc for more details. + * + *

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

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

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

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

Default is {@code 0.0%}. + */ + public Builder rangeDistributionSortKeyBaseWeight(double weight) { + writeOptions.put( + FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT.key(), Double.toString(weight)); + return this; + } + /** * Configuring the write parallel number for iceberg stream writer. * @@ -347,20 +405,23 @@ private DataStreamSink chainIcebergOperators() { flinkWriteConf = new FlinkWriteConf(table, writeOptions, readableConfig); // Find out the equality field id list based on the user-provided equality field column names. - List equalityFieldIds = checkAndGetEqualityFieldIds(); + List equalityFieldIds = + SinkUtil.checkAndGetEqualityFieldIds(table, equalityFieldColumns); - // Convert the requested flink table schema to flink row type. RowType flinkRowType = toFlinkRowType(table.schema(), tableSchema); + int writerParallelism = + flinkWriteConf.writeParallelism() == null + ? rowDataInput.getParallelism() + : flinkWriteConf.writeParallelism(); // Distribute the records from input data stream based on the write.distribution-mode and // equality fields. DataStream distributeStream = - distributeDataStream( - rowDataInput, equalityFieldIds, table.spec(), table.schema(), flinkRowType); + distributeDataStream(rowDataInput, equalityFieldIds, flinkRowType, writerParallelism); // Add parallel writers that append rows to files - SingleOutputStreamOperator writerStream = - appendWriter(distributeStream, flinkRowType, equalityFieldIds); + SingleOutputStreamOperator writerStream = + appendWriter(distributeStream, flinkRowType, equalityFieldIds, writerParallelism); // Add single-parallelism committer that commits files // after successful checkpoint or end of input @@ -426,7 +487,7 @@ private DataStreamSink appendDummySink( } private SingleOutputStreamOperator appendCommitter( - SingleOutputStreamOperator writerStream) { + SingleOutputStreamOperator writerStream) { IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter( tableLoader, @@ -446,8 +507,11 @@ private SingleOutputStreamOperator appendCommitter( return committerStream; } - private SingleOutputStreamOperator appendWriter( - DataStream input, RowType flinkRowType, List equalityFieldIds) { + private SingleOutputStreamOperator appendWriter( + DataStream input, + RowType flinkRowType, + List equalityFieldIds, + int writerParallelism) { // Validate the equality fields and partition fields if we enable the upsert mode. if (flinkWriteConf.upsertMode()) { Preconditions.checkState( @@ -481,17 +545,13 @@ private SingleOutputStreamOperator appendWriter( IcebergStreamWriter streamWriter = createStreamWriter(tableSupplier, flinkWriteConf, flinkRowType, equalityFieldIds); - int parallelism = - flinkWriteConf.writeParallelism() == null - ? input.getParallelism() - : flinkWriteConf.writeParallelism(); - SingleOutputStreamOperator writerStream = + SingleOutputStreamOperator writerStream = input .transform( operatorName(ICEBERG_STREAM_WRITER_NAME), - TypeInformation.of(WriteResult.class), + TypeInformation.of(FlinkWriteResult.class), streamWriter) - .setParallelism(parallelism); + .setParallelism(writerParallelism); if (uidPrefix != null) { writerStream = writerStream.uid(uidPrefix + "-writer"); } @@ -501,12 +561,15 @@ private SingleOutputStreamOperator appendWriter( private DataStream distributeDataStream( DataStream input, List equalityFieldIds, - PartitionSpec partitionSpec, - Schema iSchema, - RowType flinkRowType) { + RowType flinkRowType, + int writerParallelism) { DistributionMode writeMode = flinkWriteConf.distributionMode(); - LOG.info("Write distribution mode is '{}'", writeMode.modeName()); + + Schema iSchema = table.schema(); + PartitionSpec partitionSpec = table.spec(); + SortOrder sortOrder = table.sortOrder(); + switch (writeMode) { case NONE: if (equalityFieldIds.isEmpty()) { @@ -548,21 +611,52 @@ private DataStream distributeDataStream( } case RANGE: - if (equalityFieldIds.isEmpty()) { + // Ideally, exception should be thrown in the combination of range distribution and + // equality fields. Primary key case should use hash distribution mode. + // Keep the current behavior of falling back to keyBy for backward compatibility. + if (!equalityFieldIds.isEmpty()) { LOG.warn( - "Fallback to use 'none' distribution mode, because there are no equality fields set " - + "and {}=range is not supported yet in flink", - WRITE_DISTRIBUTION_MODE); - return input; - } else { - LOG.info( - "Distribute rows by equality fields, because there are equality fields set " - + "and{}=range is not supported yet in flink", + "Hash distribute rows by equality fields, even though {}=range is set. " + + "Range distribution for primary keys are not always safe in " + + "Flink streaming writer.", WRITE_DISTRIBUTION_MODE); return input.keyBy( new EqualityFieldKeySelector(iSchema, flinkRowType, equalityFieldIds)); } + // range distribute by partition key or sort key if table has an SortOrder + Preconditions.checkState( + sortOrder.isSorted() || partitionSpec.isPartitioned(), + "Invalid write distribution mode: range. Need to define sort order or partition spec."); + if (sortOrder.isUnsorted()) { + sortOrder = Partitioning.sortOrderFor(partitionSpec); + LOG.info("Construct sort order from partition spec"); + } + + LOG.info("Range distribute rows by sort order: {}", sortOrder); + StatisticsType statisticsType = flinkWriteConf.rangeDistributionStatisticsType(); + SingleOutputStreamOperator shuffleStream = + input + .transform( + operatorName("range-shuffle"), + TypeInformation.of(StatisticsOrRecord.class), + new DataStatisticsOperatorFactory( + iSchema, + sortOrder, + writerParallelism, + statisticsType, + flinkWriteConf.rangeDistributionSortKeyBaseWeight())) + // Set the parallelism same as input operator to encourage chaining + .setParallelism(input.getParallelism()); + if (uidPrefix != null) { + shuffleStream = shuffleStream.uid(uidPrefix + "-shuffle"); + } + + return shuffleStream + .partitionCustom(new RangePartitioner(iSchema, sortOrder), r -> r) + .filter(StatisticsOrRecord::hasRecord) + .map(StatisticsOrRecord::record); + default: throw new RuntimeException("Unrecognized " + WRITE_DISTRIBUTION_MODE + ": " + writeMode); } @@ -577,12 +671,9 @@ static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) { TypeUtil.validateWriteSchema(schema, writeSchema, true, true); // We use this flink schema to read values from RowData. The flink's TINYINT and SMALLINT will - // be promoted to - // iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT (backend by 1 - // 'byte'), we will - // read 4 bytes rather than 1 byte, it will mess up the byte array in BinaryRowData. So here - // we must use flink - // schema. + // be promoted to iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT + // (backend by 1 'byte'), we will read 4 bytes rather than 1 byte, it will mess up the + // byte array in BinaryRowData. So here we must use flink schema. return (RowType) requestedSchema.toRowDataType().getLogicalType(); } else { return FlinkSchemaUtil.convert(schema); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.java new file mode 100644 index 000000000000..317fb169ae1b --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.Serializable; +import org.apache.iceberg.io.WriteResult; + +public class FlinkWriteResult implements Serializable { + private final long checkpointId; + private final WriteResult writeResult; + + public FlinkWriteResult(long checkpointId, WriteResult writeResult) { + this.checkpointId = checkpointId; + this.writeResult = writeResult; + } + + public long checkpointId() { + return checkpointId; + } + + public WriteResult writeResult() { + return writeResult; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java new file mode 100644 index 000000000000..408c3e9a9d5f --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.Objects; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +/** + * The aggregated results of a single checkpoint which should be committed. Containing the + * serialized {@link org.apache.iceberg.flink.sink.DeltaManifests} file - which contains the commit + * data, and the jobId, operatorId, checkpointId triplet which helps identifying the specific commit + * + *

{@link IcebergCommittableSerializer} is used for serializing the objects between the Writer + * and the Aggregator operator and between the Aggregator and the Committer as well. + */ +class IcebergCommittable implements Serializable { + private final byte[] manifest; + private final String jobId; + private final String operatorId; + private final long checkpointId; + + IcebergCommittable(byte[] manifest, String jobId, String operatorId, long checkpointId) { + this.manifest = manifest; + this.jobId = jobId; + this.operatorId = operatorId; + this.checkpointId = checkpointId; + } + + byte[] manifest() { + return manifest; + } + + String jobId() { + return jobId; + } + + String operatorId() { + return operatorId; + } + + Long checkpointId() { + return checkpointId; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("jobId", jobId) + .add("checkpointId", checkpointId) + .add("operatorId", operatorId) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + IcebergCommittable that = (IcebergCommittable) o; + return checkpointId == that.checkpointId + && Arrays.equals(manifest, that.manifest) + && Objects.equals(jobId, that.jobId) + && Objects.equals(operatorId, that.operatorId); + } + + @Override + public int hashCode() { + int result = Objects.hash(jobId, operatorId, checkpointId); + result = 31 * result + Arrays.hashCode(manifest); + return result; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java new file mode 100644 index 000000000000..e2b388a83c75 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; + +/** + * This serializer is used for serializing the {@link IcebergCommittable} objects between the Writer + * and the Aggregator operator and between the Aggregator and the Committer as well. + * + *

In both cases only the respective part is serialized. + */ +class IcebergCommittableSerializer implements SimpleVersionedSerializer { + private static final int VERSION = 1; + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public byte[] serialize(IcebergCommittable committable) throws IOException { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + DataOutputViewStreamWrapper view = new DataOutputViewStreamWrapper(out); + view.writeUTF(committable.jobId()); + view.writeUTF(committable.operatorId()); + view.writeLong(committable.checkpointId()); + view.writeInt(committable.manifest().length); + view.write(committable.manifest()); + return out.toByteArray(); + } + + @Override + public IcebergCommittable deserialize(int version, byte[] serialized) throws IOException { + if (version == 1) { + DataInputDeserializer view = new DataInputDeserializer(serialized); + String jobId = view.readUTF(); + String operatorId = view.readUTF(); + long checkpointId = view.readLong(); + int manifestLen = view.readInt(); + byte[] manifestBuf; + manifestBuf = new byte[manifestLen]; + view.read(manifestBuf); + return new IcebergCommittable(manifestBuf, jobId, operatorId, checkpointId); + } + throw new IOException("Unrecognized version or corrupt state: " + version); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java new file mode 100644 index 000000000000..2245b36f15a7 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java @@ -0,0 +1,311 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ReplacePartitions; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.SnapshotUpdate; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class implements the Flink SinkV2 {@link Committer} interface to implement the Iceberg + * commits. The implementation builds on the following assumptions: + * + *

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

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

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

tableSupplier; + if (tableRefreshInterval != null) { + tableSupplier = new CachingTableSupplier(table, tableLoader(), tableRefreshInterval); + } else { + tableSupplier = () -> serializableTable; + } + + boolean overwriteMode = flinkWriteConf.overwriteMode(); + + // Validate the equality fields and partition fields if we enable the upsert mode. + List equalityFieldIds = + SinkUtil.checkAndGetEqualityFieldIds(table, equalityFieldColumns); + + if (flinkWriteConf.upsertMode()) { + Preconditions.checkState( + !overwriteMode, + "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); + Preconditions.checkState( + !equalityFieldIds.isEmpty(), + "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + if (!table.spec().isUnpartitioned()) { + for (PartitionField partitionField : table.spec().fields()) { + Preconditions.checkState( + equalityFieldIds.contains(partitionField.sourceId()), + "In UPSERT mode, partition field '%s' should be included in equality fields: '%s'", + partitionField, + equalityFieldColumns); + } + } + } + + return new IcebergSink( + tableLoader, + table, + snapshotSummary, + uidSuffix, + writeProperties(table, flinkWriteConf.dataFileFormat(), flinkWriteConf), + toFlinkRowType(table.schema(), tableSchema), + tableSupplier, + flinkWriteConf, + equalityFieldIds, + flinkWriteConf.branch(), + overwriteMode); + } + + /** + * Append the iceberg sink operators to write records to iceberg table. + * + * @return {@link DataStreamSink} for sink. + */ + public DataStreamSink append() { + IcebergSink sink = build(); + String suffix = defaultSuffix(uidSuffix, table.name()); + DataStream rowDataInput = inputCreator.apply(suffix); + // Please note that V2 sink framework will apply the uid here to the framework created + // operators like writer, + // committer. E.g. "Sink writer: + DataStreamSink rowDataDataStreamSink = + rowDataInput.sinkTo(sink).uid(suffix).name(suffix); + + // Note that IcebergSink internally consists o multiple operators (like writer, committer, + // aggregator). + // The following parallelism will be propagated to all of the above operators. + if (sink.flinkWriteConf.writeParallelism() != null) { + rowDataDataStreamSink.setParallelism(sink.flinkWriteConf.writeParallelism()); + } + return rowDataDataStreamSink; + } + } + + private static String defaultSuffix(String uidSuffix, String defaultSuffix) { + if (uidSuffix == null || uidSuffix.isEmpty()) { + return defaultSuffix; + } + return uidSuffix; + } + + private static SerializableTable checkAndGetTable(TableLoader tableLoader, Table table) { + if (table == null) { + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + try (TableLoader loader = tableLoader) { + return (SerializableTable) SerializableTable.copyOf(loader.loadTable()); + } catch (IOException e) { + throw new UncheckedIOException( + "Failed to load iceberg table from table loader: " + tableLoader, e); + } + } + + return (SerializableTable) SerializableTable.copyOf(table); + } + + private static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) { + if (requestedSchema != null) { + // Convert the flink schema to iceberg schema firstly, then reassign ids to match the existing + // iceberg schema. + Schema writeSchema = TypeUtil.reassignIds(FlinkSchemaUtil.convert(requestedSchema), schema); + TypeUtil.validateWriteSchema(schema, writeSchema, true, true); + + // We use this flink schema to read values from RowData. The flink's TINYINT and SMALLINT will + // be promoted to iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT + // (backend by 1 'byte'), we will read 4 bytes rather than 1 byte, it will mess up the byte + // array in BinaryRowData. So here we must use flink schema. + return (RowType) requestedSchema.toRowDataType().getLogicalType(); + } else { + return FlinkSchemaUtil.convert(schema); + } + } + + /** + * Based on the {@link FileFormat} overwrites the table level compression properties for the table + * write. + * + * @param table The table to get the table level settings + * @param format The FileFormat to use + * @param conf The write configuration + * @return The properties to use for writing + */ + private static Map writeProperties( + Table table, FileFormat format, FlinkWriteConf conf) { + Map writeProperties = Maps.newHashMap(table.properties()); + + switch (format) { + case PARQUET: + writeProperties.put(PARQUET_COMPRESSION, conf.parquetCompressionCodec()); + String parquetCompressionLevel = conf.parquetCompressionLevel(); + if (parquetCompressionLevel != null) { + writeProperties.put(PARQUET_COMPRESSION_LEVEL, parquetCompressionLevel); + } + + break; + case AVRO: + writeProperties.put(AVRO_COMPRESSION, conf.avroCompressionCodec()); + String avroCompressionLevel = conf.avroCompressionLevel(); + if (avroCompressionLevel != null) { + writeProperties.put(AVRO_COMPRESSION_LEVEL, conf.avroCompressionLevel()); + } + + break; + case ORC: + writeProperties.put(ORC_COMPRESSION, conf.orcCompressionCodec()); + writeProperties.put(ORC_COMPRESSION_STRATEGY, conf.orcCompressionStrategy()); + break; + default: + throw new IllegalArgumentException(String.format("Unknown file format %s", format)); + } + + return writeProperties; + } + + private DataStream distributeDataStream(DataStream input) { + DistributionMode mode = flinkWriteConf.distributionMode(); + Schema schema = table.schema(); + PartitionSpec spec = table.spec(); + LOG.info("Write distribution mode is '{}'", mode.modeName()); + switch (mode) { + case NONE: + if (equalityFieldIds.isEmpty()) { + return input; + } else { + LOG.info("Distribute rows by equality fields, because there are equality fields set"); + return input.keyBy(new EqualityFieldKeySelector(schema, flinkRowType, equalityFieldIds)); + } + + case HASH: + if (equalityFieldIds.isEmpty()) { + if (table.spec().isUnpartitioned()) { + LOG.warn( + "Fallback to use 'none' distribution mode, because there are no equality fields set " + + "and table is unpartitioned"); + return input; + } else { + if (BucketPartitionerUtil.hasOneBucketField(spec)) { + return input.partitionCustom( + new BucketPartitioner(spec), + new BucketPartitionKeySelector(spec, schema, flinkRowType)); + } else { + return input.keyBy(new PartitionKeySelector(spec, schema, flinkRowType)); + } + } + } else { + if (spec.isUnpartitioned()) { + LOG.info( + "Distribute rows by equality fields, because there are equality fields set " + + "and table is unpartitioned"); + return input.keyBy( + new EqualityFieldKeySelector(schema, flinkRowType, equalityFieldIds)); + } else { + for (PartitionField partitionField : spec.fields()) { + Preconditions.checkState( + equalityFieldIds.contains(partitionField.sourceId()), + "In 'hash' distribution mode with equality fields set, partition field '%s' " + + "should be included in equality fields: '%s'", + partitionField, + equalityFieldColumns); + } + return input.keyBy(new PartitionKeySelector(spec, schema, flinkRowType)); + } + } + + case RANGE: + if (equalityFieldIds.isEmpty()) { + LOG.warn( + "Fallback to use 'none' distribution mode, because there are no equality fields set " + + "and {}=range is not supported yet in flink", + WRITE_DISTRIBUTION_MODE); + return input; + } else { + LOG.info( + "Distribute rows by equality fields, because there are equality fields set " + + "and{}=range is not supported yet in flink", + WRITE_DISTRIBUTION_MODE); + return input.keyBy(new EqualityFieldKeySelector(schema, flinkRowType, equalityFieldIds)); + } + + default: + throw new RuntimeException("Unrecognized " + WRITE_DISTRIBUTION_MODE + ": " + mode); + } + } + + /** + * Initialize a {@link Builder} to export the data from generic input data stream into iceberg + * table. We use {@link RowData} inside the sink connector, so users need to provide a mapper + * function and a {@link TypeInformation} to convert those generic records to a RowData + * DataStream. + * + * @param input the generic source input data stream. + * @param mapper function to convert the generic data to {@link RowData} + * @param outputType to define the {@link TypeInformation} for the input data. + * @param the data type of records. + * @return {@link Builder} to connect the iceberg table. + */ + public static Builder builderFor( + DataStream input, MapFunction mapper, TypeInformation outputType) { + return new Builder().forMapperOutputType(input, mapper, outputType); + } + + /** + * Initialize a {@link Builder} to export the data from input data stream with {@link Row}s into + * iceberg table. We use {@link RowData} inside the sink connector, so users need to provide a + * {@link TableSchema} for builder to convert those {@link Row}s to a {@link RowData} DataStream. + * + * @param input the source input data stream with {@link Row}s. + * @param tableSchema defines the {@link TypeInformation} for input data. + * @return {@link Builder} to connect the iceberg table. + */ + public static Builder forRow(DataStream input, TableSchema tableSchema) { + return new Builder().forRow(input, tableSchema); + } + + /** + * Initialize a {@link Builder} to export the data from input data stream with {@link RowData}s + * into iceberg table. + * + * @param input the source input data stream with {@link RowData}s. + * @return {@link Builder} to connect the iceberg table. + */ + public static Builder forRowData(DataStream input) { + return new Builder().forRowData(input); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkWriter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkWriter.java new file mode 100644 index 000000000000..7234cf74020e --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkWriter.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.IOException; +import java.util.Collection; +import java.util.concurrent.TimeUnit; +import org.apache.flink.api.connector.sink2.CommittingSinkWriter; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Iceberg writer implementation for the {@link SinkWriter} interface. Used by the {@link + * org.apache.iceberg.flink.sink.IcebergSink} (SinkV2). Writes out the data to the final place, and + * emits a single {@link WriteResult} at every checkpoint for every data/delete file created by this + * writer. + */ +class IcebergSinkWriter implements CommittingSinkWriter { + private static final Logger LOG = LoggerFactory.getLogger(IcebergSinkWriter.class); + + private final String fullTableName; + private final TaskWriterFactory taskWriterFactory; + private final IcebergStreamWriterMetrics metrics; + private TaskWriter writer; + private final int subTaskId; + private final int attemptId; + + IcebergSinkWriter( + String fullTableName, + TaskWriterFactory taskWriterFactory, + IcebergStreamWriterMetrics metrics, + int subTaskId, + int attemptId) { + this.fullTableName = fullTableName; + this.taskWriterFactory = taskWriterFactory; + // Initialize the task writer factory. + taskWriterFactory.initialize(subTaskId, attemptId); + // Initialize the task writer. + this.writer = taskWriterFactory.create(); + this.metrics = metrics; + this.subTaskId = subTaskId; + this.attemptId = attemptId; + LOG.debug( + "Created Stream Writer for table {} subtask {} attemptId {}", + fullTableName, + subTaskId, + attemptId); + } + + @Override + public void write(RowData element, Context context) throws IOException, InterruptedException { + writer.write(element); + } + + @Override + public void flush(boolean endOfInput) { + // flush is used to handle flush/endOfInput, so no action is taken here. + } + + @Override + public void close() throws Exception { + if (writer != null) { + writer.close(); + } + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("tableName", fullTableName) + .add("subTaskId", subTaskId) + .add("attemptId", attemptId) + .toString(); + } + + @Override + public Collection prepareCommit() throws IOException { + long startNano = System.nanoTime(); + WriteResult result = writer.complete(); + this.writer = taskWriterFactory.create(); + metrics.updateFlushResult(result); + metrics.flushDuration(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); + LOG.debug( + "Iceberg writer subtask {} attempt {} flushed {} data files and {} delete files", + subTaskId, + attemptId, + result.dataFiles().length, + result.deleteFiles().length); + return Lists.newArrayList(result); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java index 9ea0349fb057..412d6c7081bf 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java @@ -29,10 +29,11 @@ import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -class IcebergStreamWriter extends AbstractStreamOperator - implements OneInputStreamOperator, BoundedOneInput { +class IcebergStreamWriter extends AbstractStreamOperator + implements OneInputStreamOperator, BoundedOneInput { private static final long serialVersionUID = 1L; + static final long END_INPUT_CHECKPOINT_ID = Long.MAX_VALUE; private final String fullTableName; private final TaskWriterFactory taskWriterFactory; @@ -63,7 +64,7 @@ public void open() { @Override public void prepareSnapshotPreBarrier(long checkpointId) throws Exception { - flush(); + flush(checkpointId); this.writer = taskWriterFactory.create(); } @@ -89,20 +90,20 @@ public void endInput() throws IOException { // Note that if the task is not closed after calling endInput, checkpoint may be triggered again // causing files to be sent repeatedly, the writer is marked as null after the last file is sent // to guard against duplicated writes. - flush(); + flush(END_INPUT_CHECKPOINT_ID); } @Override public String toString() { return MoreObjects.toStringHelper(this) - .add("table_name", fullTableName) - .add("subtask_id", subTaskId) - .add("attempt_id", attemptId) + .add("tableName", fullTableName) + .add("subTaskId", subTaskId) + .add("attemptId", attemptId) .toString(); } /** close all open files and emit files to downstream committer operator */ - private void flush() throws IOException { + private void flush(long checkpointId) throws IOException { if (writer == null) { return; } @@ -110,7 +111,7 @@ private void flush() throws IOException { long startNano = System.nanoTime(); WriteResult result = writer.complete(); writerMetrics.updateFlushResult(result); - output.collect(new StreamRecord<>(result)); + output.collect(new StreamRecord<>(new FlinkWriteResult(checkpointId, result))); writerMetrics.flushDuration(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); // Set writer to null to prevent duplicate flushes in the corner case of diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java new file mode 100644 index 000000000000..794ade577976 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.IOException; +import java.util.Collection; +import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableSummary; +import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Operator which aggregates the individual {@link WriteResult} objects) to a single {@link + * IcebergCommittable} per checkpoint (storing the serialized {@link + * org.apache.iceberg.flink.sink.DeltaManifests}, jobId, operatorId, checkpointId) + */ +class IcebergWriteAggregator extends AbstractStreamOperator> + implements OneInputStreamOperator< + CommittableMessage, CommittableMessage> { + private static final Logger LOG = LoggerFactory.getLogger(IcebergWriteAggregator.class); + private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; + private final Collection results; + private transient ManifestOutputFileFactory icebergManifestOutputFileFactory; + private transient Table table; + private final TableLoader tableLoader; + + IcebergWriteAggregator(TableLoader tableLoader) { + this.results = Sets.newHashSet(); + this.tableLoader = tableLoader; + } + + @Override + public void open() throws Exception { + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + String flinkJobId = getContainingTask().getEnvironment().getJobID().toString(); + String operatorId = getOperatorID().toString(); + int subTaskId = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(); + Preconditions.checkArgument( + subTaskId == 0, "The subTaskId must be zero in the IcebergWriteAggregator"); + int attemptId = getRuntimeContext().getTaskInfo().getAttemptNumber(); + this.table = tableLoader.loadTable(); + + this.icebergManifestOutputFileFactory = + FlinkManifestUtil.createOutputFileFactory( + () -> table, table.properties(), flinkJobId, operatorId, subTaskId, attemptId); + } + + @Override + public void finish() throws IOException { + prepareSnapshotPreBarrier(Long.MAX_VALUE); + } + + @Override + public void prepareSnapshotPreBarrier(long checkpointId) throws IOException { + IcebergCommittable committable = + new IcebergCommittable( + writeToManifest(results, checkpointId), + getContainingTask().getEnvironment().getJobID().toString(), + getRuntimeContext().getOperatorUniqueID(), + checkpointId); + CommittableMessage summary = + new CommittableSummary<>(0, 1, checkpointId, 1, 1, 0); + output.collect(new StreamRecord<>(summary)); + CommittableMessage message = + new CommittableWithLineage<>(committable, checkpointId, 0); + output.collect(new StreamRecord<>(message)); + LOG.info("Emitted commit message to downstream committer operator"); + results.clear(); + } + + /** + * Write all the completed data files to a newly created manifest file and return the manifest's + * avro serialized bytes. + */ + public byte[] writeToManifest(Collection writeResults, long checkpointId) + throws IOException { + if (writeResults.isEmpty()) { + return EMPTY_MANIFEST_DATA; + } + + WriteResult result = WriteResult.builder().addAll(writeResults).build(); + DeltaManifests deltaManifests = + FlinkManifestUtil.writeCompletedFiles( + result, () -> icebergManifestOutputFileFactory.create(checkpointId), table.spec()); + + return SimpleVersionedSerialization.writeVersionAndSerialize( + DeltaManifestsSerializer.INSTANCE, deltaManifests); + } + + @Override + public void processElement(StreamRecord> element) + throws Exception { + + if (element.isRecord() && element.getValue() instanceof CommittableWithLineage) { + results.add(((CommittableWithLineage) element.getValue()).getCommittable()); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java index da5e6e7627ae..30517cd38216 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink; +import java.util.Locale; import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Supplier; @@ -26,13 +27,13 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableOperations; import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Strings; class ManifestOutputFileFactory { // Users could define their own flink manifests directory by setting this value in table // properties. - static final String FLINK_MANIFEST_LOCATION = "flink.manifests.location"; - + @VisibleForTesting static final String FLINK_MANIFEST_LOCATION = "flink.manifests.location"; private final Supplier
tableSupplier; private final Map props; private final String flinkJobId; @@ -59,6 +60,7 @@ class ManifestOutputFileFactory { private String generatePath(long checkpointId) { return FileFormat.AVRO.addExtension( String.format( + Locale.ROOT, "%s-%s-%05d-%d-%d-%05d", flinkJobId, operatorUniqueId, diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java new file mode 100644 index 000000000000..7f28a50ecaa8 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class SinkUtil { + + private static final long INITIAL_CHECKPOINT_ID = -1L; + + public static final String FLINK_JOB_ID = "flink.job-id"; + + public static final String OPERATOR_ID = "flink.operator-id"; + public static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed-checkpoint-id"; + + private SinkUtil() {} + + private static final Logger LOG = LoggerFactory.getLogger(SinkUtil.class); + + static List checkAndGetEqualityFieldIds(Table table, List equalityFieldColumns) { + List equalityFieldIds = Lists.newArrayList(table.schema().identifierFieldIds()); + if (equalityFieldColumns != null && !equalityFieldColumns.isEmpty()) { + Set equalityFieldSet = Sets.newHashSetWithExpectedSize(equalityFieldColumns.size()); + for (String column : equalityFieldColumns) { + org.apache.iceberg.types.Types.NestedField field = table.schema().findField(column); + Preconditions.checkNotNull( + field, + "Missing required equality field column '%s' in table schema %s", + column, + table.schema()); + equalityFieldSet.add(field.fieldId()); + } + + if (!equalityFieldSet.equals(table.schema().identifierFieldIds())) { + LOG.warn( + "The configured equality field column IDs {} are not matched with the schema identifier field IDs" + + " {}, use job specified equality field columns as the equality fields by default.", + equalityFieldSet, + table.schema().identifierFieldIds()); + } + equalityFieldIds = Lists.newArrayList(equalityFieldSet); + } + return equalityFieldIds; + } + + static long getMaxCommittedCheckpointId( + Table table, String flinkJobId, String operatorId, String branch) { + Snapshot snapshot = table.snapshot(branch); + long lastCommittedCheckpointId = INITIAL_CHECKPOINT_ID; + + while (snapshot != null) { + Map summary = snapshot.summary(); + String snapshotFlinkJobId = summary.get(FLINK_JOB_ID); + String snapshotOperatorId = summary.get(OPERATOR_ID); + if (flinkJobId.equals(snapshotFlinkJobId) + && (snapshotOperatorId == null || snapshotOperatorId.equals(operatorId))) { + String value = summary.get(MAX_COMMITTED_CHECKPOINT_ID); + if (value != null) { + lastCommittedCheckpointId = Long.parseLong(value); + break; + } + } + Long parentSnapshotId = snapshot.parentId(); + snapshot = parentSnapshotId != null ? table.snapshot(parentSnapshotId) : null; + } + + return lastCommittedCheckpointId; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java new file mode 100644 index 000000000000..5a44373cccaa --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.util.InstantiationUtil; +import org.apache.iceberg.io.WriteResult; + +class WriteResultSerializer implements SimpleVersionedSerializer { + private static final int VERSION = 1; + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public byte[] serialize(WriteResult writeResult) throws IOException { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + DataOutputViewStreamWrapper view = new DataOutputViewStreamWrapper(out); + byte[] result = InstantiationUtil.serializeObject(writeResult); + view.write(result); + return out.toByteArray(); + } + + @Override + public WriteResult deserialize(int version, byte[] serialized) throws IOException { + if (version == 1) { + DataInputDeserializer view = new DataInputDeserializer(serialized); + byte[] resultBuf = new byte[serialized.length]; + view.read(resultBuf); + try { + return InstantiationUtil.deserializeObject( + resultBuf, IcebergCommittableSerializer.class.getClassLoader()); + } catch (ClassNotFoundException cnc) { + throw new IOException("Could not deserialize the WriteResult object", cnc); + } + } + throw new IOException("Unrecognized version or corrupt state: " + version); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java index 5525f02c873e..95c2328f032a 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java @@ -186,7 +186,9 @@ private boolean isComplete() { return subtaskSet.size() == parallelism; } - /** @return false if duplicate */ + /** + * @return false if duplicate + */ private boolean merge(int subtask, DataStatistics taskStatistics) { if (subtaskSet.contains(subtask)) { return false; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java index 4bfde7204acf..e2a282efd82e 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.sink.shuffle; import java.util.Comparator; +import java.util.Locale; import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; @@ -265,8 +266,10 @@ private void sendGlobalStatisticsToSubtasks(GlobalStatistics statistics) { } }, String.format( + Locale.ROOT, "Failed to send operator %s coordinator global data statistics for checkpoint %d", - operatorName, statistics.checkpointId())); + operatorName, + statistics.checkpointId())); } @SuppressWarnings("FutureReturnValueIgnored") @@ -290,8 +293,11 @@ private void handleRequestGlobalStatisticsEvent(int subtask, RequestGlobalStatis } }, String.format( + Locale.ROOT, "Failed to send operator %s coordinator global data statistics to requesting subtask %d for checkpoint %d", - operatorName, subtask, globalStatistics.checkpointId())); + operatorName, + subtask, + globalStatistics.checkpointId())); } else { LOG.info( "Ignore global statistics request from subtask {} as statistics not available", subtask); @@ -318,8 +324,11 @@ public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEven } }, String.format( + Locale.ROOT, "handling operator event %s from subtask %d (#%d)", - event.getClass(), subtask, attemptNumber)); + event.getClass(), + subtask, + attemptNumber)); } @Override @@ -339,7 +348,7 @@ public void checkpointCoordinator(long checkpointId, CompletableFuture r completedStatistics, completedStatisticsSerializer)); } }, - String.format("taking checkpoint %d", checkpointId)); + String.format(Locale.ROOT, "taking checkpoint %d", checkpointId)); } @Override @@ -381,7 +390,8 @@ public void subtaskReset(int subtask, long checkpointId) { this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); subtaskGateways.reset(subtask); }, - String.format("handling subtask %d recovery to checkpoint %d", subtask, checkpointId)); + String.format( + Locale.ROOT, "handling subtask %d recovery to checkpoint %d", subtask, checkpointId)); } @Override @@ -397,7 +407,7 @@ public void executionAttemptFailed(int subtask, int attemptNumber, @Nullable Thr this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); subtaskGateways.unregisterSubtaskGateway(subtask, attemptNumber); }, - String.format("handling subtask %d (#%d) failure", subtask, attemptNumber)); + String.format(Locale.ROOT, "handling subtask %d (#%d) failure", subtask, attemptNumber)); } @Override @@ -411,7 +421,10 @@ public void executionAttemptReady(int subtask, int attemptNumber, SubtaskGateway subtaskGateways.registerSubtaskGateway(gateway); }, String.format( - "making event gateway to subtask %d (#%d) available", subtask, attemptNumber)); + Locale.ROOT, + "making event gateway to subtask %d (#%d) available", + subtask, + attemptNumber)); } @VisibleForTesting diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java new file mode 100644 index 000000000000..dc147bf36d13 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; + +@Internal +public class DataStatisticsOperatorFactory extends AbstractStreamOperatorFactory + implements CoordinatedOperatorFactory, + OneInputStreamOperatorFactory { + + private final Schema schema; + private final SortOrder sortOrder; + private final int downstreamParallelism; + private final StatisticsType type; + private final double closeFileCostWeightPercentage; + + public DataStatisticsOperatorFactory( + Schema schema, + SortOrder sortOrder, + int downstreamParallelism, + StatisticsType type, + double closeFileCostWeightPercentage) { + this.schema = schema; + this.sortOrder = sortOrder; + this.downstreamParallelism = downstreamParallelism; + this.type = type; + this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; + } + + @Override + public OperatorCoordinator.Provider getCoordinatorProvider( + String operatorName, OperatorID operatorID) { + return new DataStatisticsCoordinatorProvider( + operatorName, + operatorID, + schema, + sortOrder, + downstreamParallelism, + type, + closeFileCostWeightPercentage); + } + + @SuppressWarnings("unchecked") + @Override + public > T createStreamOperator( + StreamOperatorParameters parameters) { + OperatorID operatorId = parameters.getStreamConfig().getOperatorID(); + String operatorName = parameters.getStreamConfig().getOperatorName(); + OperatorEventGateway gateway = + parameters.getOperatorEventDispatcher().getOperatorEventGateway(operatorId); + + DataStatisticsOperator rangeStatisticsOperator = + new DataStatisticsOperator( + operatorName, schema, sortOrder, gateway, downstreamParallelism, type); + + rangeStatisticsOperator.setup( + parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); + parameters + .getOperatorEventDispatcher() + .registerEventHandler(operatorId, rangeStatisticsOperator); + + return (T) rangeStatisticsOperator; + } + + @SuppressWarnings("rawtypes") + @Override + public Class getStreamOperatorClass(ClassLoader classLoader) { + return DataStatisticsOperator.class; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java index ce17e1964392..ce94bec14860 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java @@ -27,7 +27,9 @@ class RequestGlobalStatisticsEvent implements OperatorEvent { this.signature = null; } - /** @param signature hashCode of the subtask's existing global statistics */ + /** + * @param signature hashCode of the subtask's existing global statistics + */ RequestGlobalStatisticsEvent(int signature) { this.signature = signature; } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java index b82fc8250763..b3e536bdde52 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java @@ -23,6 +23,7 @@ import java.math.BigInteger; import java.nio.ByteBuffer; import java.util.List; +import java.util.Locale; import java.util.Objects; import java.util.Set; import java.util.UUID; @@ -169,7 +170,8 @@ public void serialize(SortKey record, DataOutputView target) throws IOException default: // SortKey transformation is a flattened struct without list and map throw new UnsupportedOperationException( - String.format("Field %d has unsupported field type: %s", fieldId, typeId)); + String.format( + Locale.ROOT, "Field %d has unsupported field type: %s", fieldId, typeId)); } } } @@ -238,7 +240,8 @@ public SortKey deserialize(SortKey reuse, DataInputView source) throws IOExcepti default: // SortKey transformation is a flattened struct without list and map throw new UnsupportedOperationException( - String.format("Field %d has unsupported field type: %s", fieldId, typeId)); + String.format( + Locale.ROOT, "Field %d has unsupported field type: %s", fieldId, typeId)); } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java index 91d975349b19..3beda960cec8 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.util.Iterator; +import java.util.Locale; import org.apache.flink.annotation.Internal; import org.apache.iceberg.CombinedScanTask; import org.apache.iceberg.FileScanTask; @@ -95,8 +96,11 @@ public void seek(int startingFileOffset, long startingRecordOffset) { } else { throw new IllegalStateException( String.format( + Locale.ROOT, "Invalid starting record offset %d for file %d from CombinedScanTask: %s", - startingRecordOffset, startingFileOffset, combinedTask)); + startingRecordOffset, + startingFileOffset, + combinedTask)); } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java index b1431a32dd20..b5a1ba85a6cb 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java @@ -28,6 +28,7 @@ import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.iceberg.Schema; @@ -46,6 +47,14 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.PropertyUtil; +/** + * Flink source builder for old {@link SourceFunction} implementation. + * + * @deprecated since 1.7.0, will be removed in 2.0.0. Use {@link IcebergSource} instead, which + * implement the newer FLIP-27 source interface. This class implements the old {@link + * SourceFunction} that has been marked as deprecated in Flink since Aug 2023. + */ +@Deprecated public class FlinkSource { private FlinkSource() {} @@ -124,7 +133,9 @@ public Builder setAll(Map properties) { return this; } - /** @deprecated Use {@link #setAll} instead. */ + /** + * @deprecated Use {@link #setAll} instead. + */ @Deprecated public Builder properties(Map properties) { readOptions.putAll(properties); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index ccbd0d9997ed..6461d8207122 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -27,7 +27,8 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; -import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.Source; import org.apache.flink.api.connector.source.SourceReader; @@ -37,6 +38,9 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.flink.util.Preconditions; @@ -61,10 +65,12 @@ import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer; import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator; import org.apache.iceberg.flink.source.reader.ColumnStatsWatermarkExtractor; +import org.apache.iceberg.flink.source.reader.ConverterReaderFunction; import org.apache.iceberg.flink.source.reader.IcebergSourceReader; import org.apache.iceberg.flink.source.reader.IcebergSourceReaderMetrics; import org.apache.iceberg.flink.source.reader.MetaDataReaderFunction; import org.apache.iceberg.flink.source.reader.ReaderFunction; +import org.apache.iceberg.flink.source.reader.RowDataConverter; import org.apache.iceberg.flink.source.reader.RowDataReaderFunction; import org.apache.iceberg.flink.source.reader.SerializableRecordEmitter; import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; @@ -72,13 +78,13 @@ import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.flink.source.split.SplitComparators; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.ThreadPools; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -@Experimental public class IcebergSource implements Source { private static final Logger LOG = LoggerFactory.getLogger(IcebergSource.class); @@ -95,6 +101,11 @@ public class IcebergSource implements Source emitter; private final String tableName; + // cache the discovered splits by planSplitsForBatch, which can be called twice. And they come + // from two different threads: (1) source/stream construction by main thread (2) enumerator + // creation. Hence need volatile here. + private volatile List batchSplits; + IcebergSource( TableLoader tableLoader, ScanContext scanContext, @@ -130,16 +141,26 @@ private String planningThreadName() { return tableName + "-" + UUID.randomUUID(); } + /** + * Cache the enumerated splits for batch execution to avoid double planning as there are two code + * paths obtaining splits: (1) infer parallelism (2) enumerator creation. + */ private List planSplitsForBatch(String threadName) { + if (batchSplits != null) { + return batchSplits; + } + ExecutorService workerPool = ThreadPools.newWorkerPool(threadName, scanContext.planParallelism()); try (TableLoader loader = tableLoader.clone()) { loader.open(); - List splits = + this.batchSplits = FlinkSplitPlanner.planIcebergSourceSplits(loader.loadTable(), scanContext, workerPool); LOG.info( - "Discovered {} splits from table {} during job initialization", splits.size(), tableName); - return splits; + "Discovered {} splits from table {} during job initialization", + batchSplits.size(), + tableName); + return batchSplits; } catch (IOException e) { throw new UncheckedIOException("Failed to close table loader", e); } finally { @@ -205,26 +226,69 @@ private SplitEnumerator createEnumer // Only do scan planning if nothing is restored from checkpoint state List splits = planSplitsForBatch(planningThreadName()); assigner.onDiscoveredSplits(splits); + // clear the cached splits after enumerator creation as they won't be needed anymore + this.batchSplits = null; } return new StaticIcebergEnumerator(enumContext, assigner); } } + private boolean shouldInferParallelism() { + return !scanContext.isStreaming(); + } + + private int inferParallelism(ReadableConfig flinkConf, StreamExecutionEnvironment env) { + int parallelism = + SourceUtil.inferParallelism( + flinkConf, + scanContext.limit(), + () -> { + List splits = planSplitsForBatch(planningThreadName()); + return splits.size(); + }); + + if (env.getMaxParallelism() > 0) { + parallelism = Math.min(parallelism, env.getMaxParallelism()); + } + + return parallelism; + } + + /** + * Create a source builder. + * + * @deprecated since 1.7.0. Will be removed in 2.0.0; use{@link IcebergSource#forRowData()} or + * {@link IcebergSource#forOutputType(RowDataConverter)} instead + */ + @Deprecated public static Builder builder() { return new Builder<>(); } + /** Create a source builder for RowData output type. */ public static Builder forRowData() { return new Builder<>(); } + /** + * Create a source builder that would convert {@link RowData} to the output type {@code T}. + * + * @param converter convert {@link RowData} to output type {@code T} + * @param output type + * @return an IcebergSource builder + */ + public static Builder forOutputType(RowDataConverter converter) { + return new Builder().converter(converter); + } + public static class Builder { private TableLoader tableLoader; private Table table; private SplitAssignerFactory splitAssignerFactory; private SerializableComparator splitComparator; private ReaderFunction readerFunction; + private RowDataConverter converter; private ReadableConfig flinkConfig = new Configuration(); private final ScanContext.Builder contextBuilder = ScanContext.builder(); private TableSchema projectedFlinkSchema; @@ -255,11 +319,28 @@ public Builder splitComparator( return this; } + /** + * @deprecated since 1.7.0. Will be removed in 2.0.0; use{@link + * IcebergSource#forOutputType(RowDataConverter)} instead to produce output type other than + * {@link RowData}. + */ + @Deprecated public Builder readerFunction(ReaderFunction newReaderFunction) { + Preconditions.checkState( + converter == null, + "Cannot set reader function when builder was created via IcebergSource.forOutputType(Converter)"); this.readerFunction = newReaderFunction; return this; } + /** + * Don't need to be public. It is set by {@link IcebergSource#forOutputType(RowDataConverter)}. + */ + private Builder converter(RowDataConverter newConverter) { + this.converter = newConverter; + return this; + } + public Builder flinkConfig(ReadableConfig config) { this.flinkConfig = config; return this; @@ -464,7 +545,9 @@ public Builder watermarkColumnTimeUnit(TimeUnit timeUnit) { return this; } - /** @deprecated Use {@link #setAll} instead. */ + /** + * @deprecated Use {@link #setAll} instead. + */ @Deprecated public Builder properties(Map properties) { readOptions.putAll(properties); @@ -482,6 +565,10 @@ public IcebergSource build() { } contextBuilder.resolveConfig(table, readOptions, flinkConfig); + contextBuilder.exposeLocality( + SourceUtil.isLocalityEnabled(table, flinkConfig, exposeLocality)); + contextBuilder.planParallelism( + flinkConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE)); Schema icebergSchema = table.schema(); if (projectedFlinkSchema != null) { contextBuilder.project(FlinkSchemaUtil.convert(icebergSchema, projectedFlinkSchema)); @@ -506,25 +593,7 @@ public IcebergSource build() { ScanContext context = contextBuilder.build(); context.validate(); if (readerFunction == null) { - if (table instanceof BaseMetadataTable) { - MetaDataReaderFunction rowDataReaderFunction = - new MetaDataReaderFunction( - flinkConfig, table.schema(), context.project(), table.io(), table.encryption()); - this.readerFunction = (ReaderFunction) rowDataReaderFunction; - } else { - RowDataReaderFunction rowDataReaderFunction = - new RowDataReaderFunction( - flinkConfig, - table.schema(), - context.project(), - context.nameMapping(), - context.caseSensitive(), - table.io(), - table.encryption(), - context.filters(), - context.limit()); - this.readerFunction = (ReaderFunction) rowDataReaderFunction; - } + this.readerFunction = readerFunction(context); } if (splitAssignerFactory == null) { @@ -545,5 +614,75 @@ public IcebergSource build() { table, emitter); } + + /** + * Build the {@link IcebergSource} and create a {@link DataStream} from the source. Watermark + * strategy is set to {@link WatermarkStrategy#noWatermarks()}. + * + * @return data stream from the Iceberg source + */ + public DataStream buildStream(StreamExecutionEnvironment env) { + // buildStream should only be called with RowData or Converter paths. + Preconditions.checkState( + readerFunction == null, + "Cannot set reader function when building a data stream from the source"); + IcebergSource source = build(); + TypeInformation outputTypeInfo = + outputTypeInfo(converter, table.schema(), source.scanContext.project()); + DataStreamSource stream = + env.fromSource(source, WatermarkStrategy.noWatermarks(), source.name(), outputTypeInfo); + if (source.shouldInferParallelism()) { + stream = stream.setParallelism(source.inferParallelism(flinkConfig, env)); + } + + return stream; + } + + private static TypeInformation outputTypeInfo( + RowDataConverter converter, Schema tableSchema, Schema projected) { + if (converter != null) { + return converter.getProducedType(); + } else { + // output type is RowData + Schema readSchema = projected != null ? projected : tableSchema; + return (TypeInformation) + FlinkCompatibilityUtil.toTypeInfo(FlinkSchemaUtil.convert(readSchema)); + } + } + + private ReaderFunction readerFunction(ScanContext context) { + if (table instanceof BaseMetadataTable) { + MetaDataReaderFunction rowDataReaderFunction = + new MetaDataReaderFunction( + flinkConfig, table.schema(), context.project(), table.io(), table.encryption()); + return (ReaderFunction) rowDataReaderFunction; + } else { + if (converter == null) { + return (ReaderFunction) + new RowDataReaderFunction( + flinkConfig, + table.schema(), + context.project(), + context.nameMapping(), + context.caseSensitive(), + table.io(), + table.encryption(), + context.filters(), + context.limit()); + } else { + return new ConverterReaderFunction<>( + converter, + flinkConfig, + table.schema(), + context.project(), + context.nameMapping(), + context.caseSensitive(), + table.io(), + table.encryption(), + context.filters(), + context.limit()); + } + } + } } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java index 610657e8d47b..65adce77d9f9 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java @@ -23,11 +23,8 @@ import java.util.Map; import java.util.Optional; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.connector.ChangelogMode; @@ -128,26 +125,18 @@ private DataStream createDataStream(StreamExecutionEnvironment execEnv) .build(); } - private DataStreamSource createFLIP27Stream(StreamExecutionEnvironment env) { + private DataStream createFLIP27Stream(StreamExecutionEnvironment env) { SplitAssignerType assignerType = readableConfig.get(FlinkConfigOptions.TABLE_EXEC_SPLIT_ASSIGNER_TYPE); - IcebergSource source = - IcebergSource.forRowData() - .tableLoader(loader) - .assignerFactory(assignerType.factory()) - .properties(properties) - .project(getProjectedSchema()) - .limit(limit) - .filters(filters) - .flinkConfig(readableConfig) - .build(); - DataStreamSource stream = - env.fromSource( - source, - WatermarkStrategy.noWatermarks(), - source.name(), - TypeInformation.of(RowData.class)); - return stream; + return IcebergSource.forRowData() + .tableLoader(loader) + .assignerFactory(assignerType.factory()) + .properties(properties) + .project(getProjectedSchema()) + .limit(limit) + .filters(filters) + .flinkConfig(readableConfig) + .buildStream(env); } private TableSchema getProjectedSchema() { diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java index c958604c004a..391633924264 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java @@ -104,10 +104,7 @@ public static class RewriteMap extends RichMapFunction taskWriterFactory; private final RowDataFileScanTaskReader rowDataReader; @@ -119,10 +116,7 @@ public RewriteMap( boolean caseSensitive, EncryptionManager encryptionManager, TaskWriterFactory taskWriterFactory) { - this.schema = schema; - this.nameMapping = nameMapping; this.io = io; - this.caseSensitive = caseSensitive; this.encryptionManager = encryptionManager; this.taskWriterFactory = taskWriterFactory; this.rowDataReader = diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java index 280a126a46ce..fc310606dee9 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java @@ -22,6 +22,7 @@ import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicReference; @@ -76,6 +77,7 @@ public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname // Iceberg source uses custom split request event to piggyback finished split ids. throw new UnsupportedOperationException( String.format( + Locale.ROOT, "Received invalid default split request event " + "from subtask %d as Iceberg source uses custom split request event", subtaskId)); @@ -92,8 +94,10 @@ public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { } else { throw new IllegalArgumentException( String.format( + Locale.ROOT, "Received unknown event from subtask %d: %s", - subtaskId, sourceEvent.getClass().getCanonicalName())); + subtaskId, + sourceEvent.getClass().getCanonicalName())); } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java index c7021b9c6847..c50c3854ee14 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java @@ -37,6 +37,7 @@ public class ContinuousIcebergEnumerator extends AbstractIcebergEnumerator { private static final Logger LOG = LoggerFactory.getLogger(ContinuousIcebergEnumerator.class); + /** * This is hardcoded, as {@link ScanContext#maxPlanningSnapshotCount()} could be the knob to * control the total number of snapshots worth of splits tracked by assigner. diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.java new file mode 100644 index 000000000000..b158b0871a53 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.avro.RowDataToAvroConverters; +import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; +import org.apache.flink.formats.avro.typeutils.GenericRecordAvroTypeInfo; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.flink.FlinkSchemaUtil; + +public class AvroGenericRecordConverter implements RowDataConverter { + private final Schema avroSchema; + private final RowDataToAvroConverters.RowDataToAvroConverter flinkConverter; + private final TypeInformation outputTypeInfo; + + private AvroGenericRecordConverter(Schema avroSchema, RowType rowType) { + this.avroSchema = avroSchema; + this.flinkConverter = RowDataToAvroConverters.createConverter(rowType); + this.outputTypeInfo = new GenericRecordAvroTypeInfo(avroSchema); + } + + public static AvroGenericRecordConverter fromIcebergSchema( + org.apache.iceberg.Schema icebergSchema, String tableName) { + RowType rowType = FlinkSchemaUtil.convert(icebergSchema); + Schema avroSchema = AvroSchemaUtil.convert(icebergSchema, tableName); + return new AvroGenericRecordConverter(avroSchema, rowType); + } + + public static AvroGenericRecordConverter fromAvroSchema(Schema avroSchema, String tableName) { + DataType dataType = AvroSchemaConverter.convertToDataType(avroSchema.toString()); + LogicalType logicalType = TypeConversions.fromDataToLogicalType(dataType); + RowType rowType = RowType.of(logicalType.getChildren().toArray(new LogicalType[0])); + return new AvroGenericRecordConverter(avroSchema, rowType); + } + + @Override + public GenericRecord apply(RowData rowData) { + return (GenericRecord) flinkConverter.convert(avroSchema, rowData); + } + + @Override + public TypeInformation getProducedType() { + return outputTypeInfo; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java index 66e59633fff2..f89e5ce13474 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java @@ -28,13 +28,21 @@ import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.flink.source.AvroGenericRecordFileScanTaskReader; import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.IcebergSource; import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; import org.apache.iceberg.flink.source.RowDataToAvroGenericRecordConverter; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -/** Read Iceberg rows as {@link GenericRecord}. */ +/** + * Read Iceberg rows as {@link GenericRecord}. + * + * @deprecated since 1.7.0. Will be removed in 2.0.0; use {@link + * IcebergSource#forOutputType(RowDataConverter)} and {@link AvroGenericRecordConverter} + * instead. + */ +@Deprecated public class AvroGenericRecordReaderFunction extends DataIteratorReaderFunction { private final String tableName; private final Schema readSchema; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java new file mode 100644 index 000000000000..e1e7c17d63c5 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import java.util.List; +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Schema; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.InputFilesDecryptor; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.FileScanTaskReader; +import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +@Internal +public class ConverterReaderFunction extends DataIteratorReaderFunction { + private final RowDataConverter converter; + private final Schema tableSchema; + private final Schema readSchema; + private final String nameMapping; + private final boolean caseSensitive; + private final FileIO io; + private final EncryptionManager encryption; + private final List filters; + private final long limit; + + private transient RecordLimiter recordLimiter = null; + + public ConverterReaderFunction( + RowDataConverter converter, + ReadableConfig config, + Schema tableSchema, + Schema projectedSchema, + String nameMapping, + boolean caseSensitive, + FileIO io, + EncryptionManager encryption, + List filters, + long limit) { + super(new ListDataIteratorBatcher<>(config)); + this.converter = converter; + this.tableSchema = tableSchema; + this.readSchema = readSchema(tableSchema, projectedSchema); + this.nameMapping = nameMapping; + this.caseSensitive = caseSensitive; + this.io = io; + this.encryption = encryption; + this.filters = filters; + this.limit = limit; + } + + @Override + protected DataIterator createDataIterator(IcebergSourceSplit split) { + RowDataFileScanTaskReader rowDataReader = + new RowDataFileScanTaskReader(tableSchema, readSchema, nameMapping, caseSensitive, filters); + return new LimitableDataIterator<>( + new ConverterFileScanTaskReader<>(rowDataReader, converter), + split.task(), + io, + encryption, + lazyLimiter()); + } + + private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { + Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); + return projectedSchema == null ? tableSchema : projectedSchema; + } + + /** Lazily create RecordLimiter to avoid the need to make it serializable */ + private RecordLimiter lazyLimiter() { + if (recordLimiter == null) { + this.recordLimiter = RecordLimiter.create(limit); + } + + return recordLimiter; + } + + private static class ConverterFileScanTaskReader implements FileScanTaskReader { + private final RowDataFileScanTaskReader rowDataReader; + private final RowDataConverter converter; + + ConverterFileScanTaskReader( + RowDataFileScanTaskReader rowDataReader, RowDataConverter converter) { + this.rowDataReader = rowDataReader; + this.converter = converter; + } + + @Override + public CloseableIterator open( + FileScanTask fileScanTask, InputFilesDecryptor inputFilesDecryptor) { + return CloseableIterator.transform( + rowDataReader.open(fileScanTask, inputFilesDecryptor), converter); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java index 9c20494fdbcd..bcd72e25036b 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java @@ -85,7 +85,7 @@ public RecordsWithSplitIds> fetch() throws IOException { } else { // return an empty result, which will lead to split fetch to be idle. // SplitFetcherManager will then close idle fetcher. - return new RecordsBySplits(Collections.emptyMap(), Collections.emptySet()); + return new RecordsBySplits<>(Collections.emptyMap(), Collections.emptySet()); } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java index 6ac92592b6aa..10e7d2037a30 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.source.reader; +import java.util.Locale; import org.apache.flink.annotation.Internal; /** @@ -73,6 +74,6 @@ public void record(T nextRecord) { @Override public String toString() { - return String.format("%s @ %d + %d", record, fileOffset, recordOffset); + return String.format(Locale.ROOT, "%s @ %d + %d", record, fileOffset, recordOffset); } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java new file mode 100644 index 000000000000..98bb7e981840 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import java.io.Serializable; +import java.util.function.Function; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.table.data.RowData; + +/** + * Convert RowData to a different output type. + * + * @param output type + */ +public interface RowDataConverter + extends Function, ResultTypeQueryable, Serializable {} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java index d4b0f9e1977d..d90d1dc88c91 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.source.split; import java.io.IOException; +import java.util.Locale; import org.apache.flink.annotation.Internal; import org.apache.flink.core.io.SimpleVersionedSerializer; @@ -54,6 +55,7 @@ public IcebergSourceSplit deserialize(int version, byte[] serialized) throws IOE default: throw new IOException( String.format( + Locale.ROOT, "Failed to deserialize IcebergSourceSplit. " + "Encountered unsupported version: %d. Supported version are [1]", version)); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java index 482cfd110bde..b63547d433a4 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.flink.FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HADOOP; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; import java.util.Arrays; @@ -46,6 +47,7 @@ import org.apache.iceberg.Parameter; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Parameters; +import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.Namespace; @@ -54,6 +56,7 @@ import org.apache.iceberg.hadoop.HadoopCatalog; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -241,4 +244,93 @@ public void testHashDistributeMode() throws Exception { sql("DROP TABLE IF EXISTS %s.%s", FLINK_DATABASE, tableName); } } + + @TestTemplate + public void testRangeDistributionPartitionColumn() { + // Range partitioner currently only works with streaming writes (with checkpoints) + assumeThat(isStreamingJob).isTrue(); + + // Initialize a BoundedSource table to precisely emit those rows in only one checkpoint. + List> rowsPerCheckpoint = + IntStream.range(1, 6) + .mapToObj( + checkpointId -> { + List charRows = Lists.newArrayList(); + // emit 26x10 rows for each checkpoint cycle + for (int i = 0; i < 10; ++i) { + for (char c = 'a'; c <= 'z'; c++) { + charRows.add(Row.of(c - 'a', String.valueOf(c))); + } + } + return charRows; + }) + .collect(Collectors.toList()); + List flattenedRows = + rowsPerCheckpoint.stream().flatMap(List::stream).collect(Collectors.toList()); + + String dataId = BoundedTableFactory.registerDataSet(rowsPerCheckpoint); + sql( + "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" + + " WITH ('connector'='BoundedSource', 'data-id'='%s')", + SOURCE_TABLE, dataId); + + assertThat(sql("SELECT * FROM %s", SOURCE_TABLE)) + .as("Should have the expected rows in source table.") + .containsExactlyInAnyOrderElementsOf(flattenedRows); + + Map tableProps = + ImmutableMap.of( + "write.format.default", + FileFormat.PARQUET.name(), + TableProperties.WRITE_DISTRIBUTION_MODE, + DistributionMode.RANGE.modeName()); + + String tableName = "test_hash_distribution_mode"; + sql( + "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH %s", + tableName, toWithClause(tableProps)); + + try { + // Insert data set. + sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); + + assertThat(sql("SELECT * FROM %s", tableName)) + .as("Should have the expected rows in sink table.") + .containsExactlyInAnyOrderElementsOf(flattenedRows); + + Table table = catalog.loadTable(TableIdentifier.of(ICEBERG_NAMESPACE, tableName)); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(5); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + for (Snapshot snapshot : rangePartitionedCycles) { + List addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // range partition results in each partition only assigned to one writer task + // maybe less than 26 partitions as BoundedSource doesn't always precisely + // control the checkpoint boundary. + // It is hard to precisely control the test condition in SQL tests. + // Here only minimal safe assertions are applied to avoid flakiness. + // If there are no shuffling, the number of data files could be as high as + // 26 * 4 as the default parallelism is set to 4 for the mini cluster. + assertThat(addedDataFiles).hasSizeLessThanOrEqualTo(26); + } + } finally { + sql("DROP TABLE IF EXISTS %s.%s", FLINK_DATABASE, tableName); + } + } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ConstantsForTests.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ConstantsForTests.java new file mode 100644 index 000000000000..36e162d4f068 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ConstantsForTests.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +class ConstantsForTests { + public static final long EVENT_TIME = 10L; + static final long EVENT_TIME_2 = 11L; + static final String DUMMY_NAME = "dummy"; + + private ConstantsForTests() { + // Do not instantiate + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ManualSource.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ManualSource.java index 38bb9c393fa9..679b3ec508a2 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ManualSource.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ManualSource.java @@ -22,6 +22,7 @@ import java.util.Collections; import java.util.List; import java.util.concurrent.CompletableFuture; +import org.apache.flink.api.common.eventtime.Watermark; import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.Boundedness; @@ -156,23 +157,26 @@ public SimpleVersionedSerializer getEnumeratorCheckpointSeriali @Override public SourceReader createReader(SourceReaderContext sourceReaderContext) { - return new SourceReader() { + return new SourceReader<>() { @Override public void start() { // Do nothing } + @SuppressWarnings("unchecked") @Override public InputStatus pollNext(ReaderOutput output) { Tuple2 next = (Tuple2) QUEUES.get(index).poll(); if (next != null) { if (next.f0 == null) { - // No more input - return InputStatus.END_OF_INPUT; - } - - if (next.f1 == null) { + if (next.f1 == null) { + // No more input + return InputStatus.END_OF_INPUT; + } else { + output.emitWatermark(new Watermark(next.f1)); + } + } else if (next.f1 == null) { // No event time set output.collect(next.f0); } else { diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java new file mode 100644 index 000000000000..7a523035b7fb --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java @@ -0,0 +1,153 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Arrays; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricConfig; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.reporter.MetricReporter; +import org.apache.flink.metrics.reporter.MetricReporterFactory; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +public class MetricsReporterFactoryForTests implements MetricReporterFactory { + private static final TestMetricsReporter INSTANCE = new TestMetricsReporter(); + private static final Pattern FULL_METRIC_NAME = + Pattern.compile( + "\\.taskmanager\\.[^.]+\\.[^.]+\\.([^.]+)\\.\\d+\\." + + TableMaintenanceMetrics.GROUP_KEY + + "\\.([^.]+)\\.([^.]+)"); + + private static Map counters = Maps.newConcurrentMap(); + private static Map gauges = Maps.newConcurrentMap(); + private static Set monitoredMetricNames; + + public MetricsReporterFactoryForTests() { + monitoredMetricNames = + Arrays.stream(TableMaintenanceMetrics.class.getDeclaredFields()) + .map( + f -> { + try { + return f.get(null).toString(); + } catch (IllegalAccessException e) { + throw new RuntimeException(e); + } + }) + .collect(Collectors.toSet()); + } + + @Override + public MetricReporter createMetricReporter(Properties properties) { + return INSTANCE; + } + + public static void reset() { + counters = Maps.newConcurrentMap(); + gauges = Maps.newConcurrentMap(); + } + + public static Long counter(String name) { + return counterValues().get(name); + } + + public static Long gauge(String name) { + return gaugeValues().get(name); + } + + public static void assertGauges(Map expected) { + assertThat(filter(gaugeValues(), expected)).isEqualTo(filter(expected, expected)); + } + + public static void assertCounters(Map expected) { + assertThat(filter(counterValues(), expected)).isEqualTo(filter(expected, expected)); + } + + private static Map gaugeValues() { + return gauges.entrySet().stream() + .collect( + Collectors.toMap( + entry -> longName(entry.getKey()), entry -> (Long) entry.getValue().getValue())); + } + + private static Map counterValues() { + return counters.entrySet().stream() + .collect( + Collectors.toMap( + entry -> longName(entry.getKey()), entry -> entry.getValue().getCount())); + } + + private static Map filter(Map original, Map filter) { + return original.entrySet().stream() + .filter( + entry -> { + Long filterValue = filter.get(entry.getKey()); + return filterValue == null || filterValue != -1; + }) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + } + + private static String longName(String fullName) { + Matcher matcher = FULL_METRIC_NAME.matcher(fullName); + if (!matcher.matches()) { + throw new RuntimeException(String.format("Can't parse simplified metrics name %s", fullName)); + } + + return matcher.group(1) + "." + matcher.group(2) + "." + matcher.group(3); + } + + private static class TestMetricsReporter implements MetricReporter { + @Override + public void open(MetricConfig config) { + // do nothing + } + + @Override + public void close() { + // do nothing + } + + @Override + public void notifyOfAddedMetric(Metric metric, String metricName, MetricGroup group) { + if (monitoredMetricNames.contains(metricName)) { + if (metric instanceof Counter) { + counters.put(group.getMetricIdentifier(metricName), (Counter) metric); + } + + if (metric instanceof Gauge) { + gauges.put(group.getMetricIdentifier(metricName), (Gauge) metric); + } + } + } + + @Override + public void notifyOfRemovedMetric(Metric metric, String metricName, MetricGroup group) { + // do nothing + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java index 272e0b693fd3..225853086545 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java @@ -20,16 +20,24 @@ import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; +import java.io.File; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.MetricOptions; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.core.execution.SavepointFormatType; +import org.apache.flink.runtime.jobgraph.SavepointConfigOptions; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.iceberg.flink.FlinkCatalogFactory; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.awaitility.Awaitility; import org.junit.jupiter.api.extension.RegisterExtension; class OperatorTestBase { private static final int NUMBER_TASK_MANAGERS = 1; private static final int SLOTS_PER_TASK_MANAGER = 8; + private static final TriggerLockFactory.Lock MAINTENANCE_LOCK = new MemoryLock(); + private static final TriggerLockFactory.Lock RECOVERY_LOCK = new MemoryLock(); static final String TABLE_NAME = "test_table"; @@ -39,7 +47,7 @@ class OperatorTestBase { new MiniClusterResourceConfiguration.Builder() .setNumberTaskManagers(NUMBER_TASK_MANAGERS) .setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER) - .setConfiguration(new Configuration(DISABLE_CLASSLOADER_CHECK_CONFIG)) + .setConfiguration(config()) .build()); @RegisterExtension @@ -48,4 +56,101 @@ class OperatorTestBase { "catalog", ImmutableMap.of("type", "iceberg", FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hadoop"), "db"); + + private static Configuration config() { + Configuration config = new Configuration(DISABLE_CLASSLOADER_CHECK_CONFIG); + MetricOptions.forReporter(config, "test_reporter") + .set(MetricOptions.REPORTER_FACTORY_CLASS, MetricsReporterFactoryForTests.class.getName()); + return config; + } + + protected static TriggerLockFactory lockFactory() { + return new TriggerLockFactory() { + @Override + public void open() { + MAINTENANCE_LOCK.unlock(); + RECOVERY_LOCK.unlock(); + } + + @Override + public Lock createLock() { + return MAINTENANCE_LOCK; + } + + @Override + public Lock createRecoveryLock() { + return RECOVERY_LOCK; + } + + @Override + public void close() { + // do nothing + } + }; + } + + /** + * Close the {@link JobClient} and wait for the job closure. If the savepointDir is specified, it + * stops the job with a savepoint. + * + * @param jobClient the job to close + * @param savepointDir the savepointDir to store the last savepoint. If null then + * stop without a savepoint. + * @return configuration for restarting the job from the savepoint + */ + public static Configuration closeJobClient(JobClient jobClient, File savepointDir) { + Configuration conf = new Configuration(); + if (jobClient != null) { + if (savepointDir != null) { + // Stop with savepoint + jobClient.stopWithSavepoint(false, savepointDir.getPath(), SavepointFormatType.CANONICAL); + // Wait until the savepoint is created and the job has been stopped + Awaitility.await().until(() -> savepointDir.listFiles(File::isDirectory).length == 1); + conf.set( + SavepointConfigOptions.SAVEPOINT_PATH, + savepointDir.listFiles(File::isDirectory)[0].getAbsolutePath()); + } else { + jobClient.cancel(); + } + + // Wait until the job has been stopped + Awaitility.await().until(() -> jobClient.getJobStatus().get().isTerminalState()); + return conf; + } + + return null; + } + + /** + * Close the {@link JobClient} and wait for the job closure. + * + * @param jobClient the job to close + */ + public static void closeJobClient(JobClient jobClient) { + closeJobClient(jobClient, null); + } + + private static class MemoryLock implements TriggerLockFactory.Lock { + boolean locked = false; + + @Override + public boolean tryLock() { + if (locked) { + return false; + } else { + locked = true; + return true; + } + } + + @Override + public boolean isHeld() { + return locked; + } + + @Override + public void unlock() { + locked = false; + } + } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestJdbcLockFactory.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestJdbcLockFactory.java new file mode 100644 index 000000000000..051d09d92bad --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestJdbcLockFactory.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import static org.apache.iceberg.flink.maintenance.operator.JdbcLockFactory.INIT_LOCK_TABLES_PROPERTY; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.jdbc.JdbcCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.Test; + +class TestJdbcLockFactory extends TestLockFactoryBase { + @Override + TriggerLockFactory lockFactory() { + return lockFactory("tableName"); + } + + @Test + void testMultiTableLock() { + JdbcLockFactory other = lockFactory("tableName2"); + other.open((JdbcLockFactory) this.lockFactory); + TriggerLockFactory.Lock lock1 = lockFactory.createLock(); + TriggerLockFactory.Lock lock2 = other.createLock(); + assertThat(lock1.tryLock()).isTrue(); + assertThat(lock2.tryLock()).isTrue(); + } + + private JdbcLockFactory lockFactory(String tableName) { + Map properties = Maps.newHashMap(); + properties.put(JdbcCatalog.PROPERTY_PREFIX + "username", "user"); + properties.put(JdbcCatalog.PROPERTY_PREFIX + "password", "password"); + properties.put(INIT_LOCK_TABLES_PROPERTY, "true"); + + return new JdbcLockFactory( + "jdbc:sqlite:file::memory:?ic" + UUID.randomUUID().toString().replace("-", ""), + tableName, + properties); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockFactoryBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockFactoryBase.java new file mode 100644 index 000000000000..bf9e86f2534d --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockFactoryBase.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +abstract class TestLockFactoryBase { + protected TriggerLockFactory lockFactory; + + abstract TriggerLockFactory lockFactory(); + + @BeforeEach + void before() { + this.lockFactory = lockFactory(); + lockFactory.open(); + } + + @AfterEach + void after() throws IOException { + lockFactory.close(); + } + + @Test + void testTryLock() { + TriggerLockFactory.Lock lock1 = lockFactory.createLock(); + TriggerLockFactory.Lock lock2 = lockFactory.createLock(); + assertThat(lock1.tryLock()).isTrue(); + assertThat(lock1.tryLock()).isFalse(); + assertThat(lock2.tryLock()).isFalse(); + } + + @Test + void testUnLock() { + TriggerLockFactory.Lock lock = lockFactory.createLock(); + assertThat(lock.tryLock()).isTrue(); + + lock.unlock(); + assertThat(lock.tryLock()).isTrue(); + } + + @Test + void testNoConflictWithRecoveryLock() { + TriggerLockFactory.Lock lock1 = lockFactory.createLock(); + TriggerLockFactory.Lock lock2 = lockFactory.createRecoveryLock(); + assertThat(lock1.tryLock()).isTrue(); + assertThat(lock2.tryLock()).isTrue(); + } + + @Test + void testDoubleUnLock() { + TriggerLockFactory.Lock lock = lockFactory.createLock(); + assertThat(lock.tryLock()).isTrue(); + + lock.unlock(); + lock.unlock(); + assertThat(lock.tryLock()).isTrue(); + assertThat(lock.tryLock()).isFalse(); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java new file mode 100644 index 000000000000..cffcc4eb0471 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java @@ -0,0 +1,378 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import static org.apache.iceberg.flink.maintenance.operator.ConstantsForTests.DUMMY_NAME; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.FAILED_TASK_COUNTER; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.SUCCEEDED_TASK_COUNTER; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.util.Collection; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.api.connector.sink2.CommitterInitContext; +import org.apache.flink.api.connector.sink2.CommittingSinkWriter; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.api.connector.sink2.SupportsCommitter; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; +import org.apache.flink.streaming.api.connector.sink2.SupportsPostCommitTopology; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.util.Collector; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.io.TempDir; + +@Timeout(value = 10) +class TestLockRemover extends OperatorTestBase { + private static final String[] TASKS = new String[] {"task0", "task1"}; + private static final TriggerLockFactory.Lock LOCK = new TestingLock(); + private static final TriggerLockFactory.Lock RECOVERY_LOCK = new TestingLock(); + + @TempDir private File checkpointDir; + + @BeforeEach + void before() { + MetricsReporterFactoryForTests.reset(); + } + + @Test + void testProcess() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource source = new ManualSource<>(env, TypeInformation.of(TaskResult.class)); + source + .dataStream() + .transform( + DUMMY_NAME, + TypeInformation.of(Void.class), + new LockRemover(new TestingLockFactory(), Lists.newArrayList(TASKS))) + .setParallelism(1); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + LOCK.tryLock(); + assertThat(LOCK.isHeld()).isTrue(); + + // Start a successful trigger for task1 and assert the return value is correct + processAndCheck(source, new TaskResult(0, 0L, true, Lists.newArrayList())); + + // Assert that the lock is removed + assertThat(LOCK.isHeld()).isFalse(); + } finally { + closeJobClient(jobClient); + } + } + + @Test + void testInSink() throws Exception { + String sinkName = "TestSink"; + Configuration config = new Configuration(); + config.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem"); + config.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, "file://" + checkpointDir.getPath()); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(config); + env.enableCheckpointing(10); + ManualSource source = new ManualSource<>(env, TypeInformation.of(TaskResult.class)); + source.dataStream().global().sinkTo(new SinkTest()).name(sinkName).setParallelism(1); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + LOCK.tryLock(); + assertThat(LOCK.isHeld()).isTrue(); + + // Start a successful trigger for task1 and assert the return value is correct + processAndCheck(source, new TaskResult(0, 0L, true, Lists.newArrayList()), sinkName + ": "); + + // Assert that the lock is removed + assertThat(LOCK.isHeld()).isFalse(); + } finally { + closeJobClient(jobClient); + } + } + + @Test + void testMetrics() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource source = new ManualSource<>(env, TypeInformation.of(TaskResult.class)); + source + .dataStream() + .transform( + DUMMY_NAME, + TypeInformation.of(Void.class), + new LockRemover(new TestingLockFactory(), Lists.newArrayList(TASKS))) + .setParallelism(1); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + // Start the 2 successful and one failed result trigger for task1, and 3 successful for task2 + processAndCheck(source, new TaskResult(0, 0L, true, Lists.newArrayList())); + processAndCheck(source, new TaskResult(1, 1L, true, Lists.newArrayList())); + processAndCheck(source, new TaskResult(1, 2L, true, Lists.newArrayList())); + processAndCheck(source, new TaskResult(0, 3L, false, Lists.newArrayList())); + processAndCheck(source, new TaskResult(0, 4L, true, Lists.newArrayList())); + processAndCheck(source, new TaskResult(1, 5L, true, Lists.newArrayList())); + + Awaitility.await() + .until( + () -> + MetricsReporterFactoryForTests.counter( + DUMMY_NAME + "." + TASKS[1] + "." + SUCCEEDED_TASK_COUNTER) + .equals(3L)); + + // Final check all the counters + MetricsReporterFactoryForTests.assertCounters( + new ImmutableMap.Builder() + .put(DUMMY_NAME + "." + TASKS[0] + "." + SUCCEEDED_TASK_COUNTER, 2L) + .put(DUMMY_NAME + "." + TASKS[0] + "." + FAILED_TASK_COUNTER, 1L) + .put(DUMMY_NAME + "." + TASKS[1] + "." + SUCCEEDED_TASK_COUNTER, 3L) + .put(DUMMY_NAME + "." + TASKS[1] + "." + FAILED_TASK_COUNTER, 0L) + .build()); + } finally { + closeJobClient(jobClient); + } + } + + /** + * The test checks if the recovery watermark is only removed if the watermark has arrived from + * both upstream sources. + * + * @throws Exception if any + */ + @Test + void testRecovery() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource source1 = + new ManualSource<>(env, TypeInformation.of(TaskResult.class)); + ManualSource source2 = + new ManualSource<>(env, TypeInformation.of(TaskResult.class)); + source1 + .dataStream() + .union(source2.dataStream()) + .transform( + DUMMY_NAME, + TypeInformation.of(Void.class), + new LockRemover(new TestingLockFactory(), Lists.newArrayList(TASKS[0]))) + .setParallelism(1); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + RECOVERY_LOCK.tryLock(); + assertThat(RECOVERY_LOCK.isHeld()).isTrue(); + + processAndCheck(source1, new TaskResult(0, 0L, true, Lists.newArrayList())); + + source1.sendRecord(new TaskResult(0, 1L, true, Lists.newArrayList())); + // we receive the second result - this will not happen in real use cases, but with this we can + // be sure that the previous watermark is processed + Awaitility.await() + .until( + () -> + MetricsReporterFactoryForTests.counter( + DUMMY_NAME + "." + TASKS[0] + "." + SUCCEEDED_TASK_COUNTER) + .equals(2L)); + + // We did not remove the recovery lock, as no watermark received from the other source + assertThat(RECOVERY_LOCK.isHeld()).isTrue(); + + // Recovery arrives + source1.sendWatermark(10L); + source2.sendWatermark(10L); + + Awaitility.await().until(() -> !RECOVERY_LOCK.isHeld()); + } finally { + closeJobClient(jobClient); + } + } + + private void processAndCheck(ManualSource source, TaskResult input) { + processAndCheck(source, input, null); + } + + private void processAndCheck( + ManualSource source, TaskResult input, String counterPrefix) { + source.sendRecord(input); + source.sendWatermark(input.startEpoch()); + + String counterName = + (counterPrefix != null ? counterPrefix : "") + .concat( + input.success() + ? DUMMY_NAME + "." + TASKS[input.taskIndex()] + "." + SUCCEEDED_TASK_COUNTER + : DUMMY_NAME + "." + TASKS[input.taskIndex()] + "." + FAILED_TASK_COUNTER); + Long counterValue = MetricsReporterFactoryForTests.counter(counterName); + Long expected = counterValue != null ? counterValue + 1 : 1L; + + Awaitility.await() + .until(() -> expected.equals(MetricsReporterFactoryForTests.counter(counterName))); + } + + private static class TestingLockFactory implements TriggerLockFactory { + @Override + public void open() { + // Do nothing + } + + @Override + public Lock createLock() { + return LOCK; + } + + @Override + public Lock createRecoveryLock() { + return RECOVERY_LOCK; + } + + @Override + public void close() { + // Do nothing + } + } + + private static class TestingLock implements TriggerLockFactory.Lock { + private boolean locked = false; + + @Override + public boolean tryLock() { + if (isHeld()) { + return false; + } else { + locked = true; + return true; + } + } + + @Override + public boolean isHeld() { + return locked; + } + + @Override + public void unlock() { + locked = false; + } + } + + private static class SinkTest + implements Sink, + SupportsCommitter, + SupportsPostCommitTopology { + @Override + public SinkWriter createWriter(InitContext initContext) { + return new CommittingSinkWriter() { + private final Collection received = Lists.newArrayList(); + + @Override + public Collection prepareCommit() { + Collection result = Lists.newArrayList(received); + received.clear(); + return result; + } + + @Override + public void write(TaskResult taskResult, Context context) { + received.add(taskResult); + } + + @Override + public void flush(boolean b) { + // noop + } + + @Override + public void close() { + // noop + } + }; + } + + @Override + public Committer createCommitter(CommitterInitContext committerInitContext) { + return new Committer<>() { + @Override + public void commit(Collection> collection) { + // noop + } + + @Override + public void close() { + // noop + } + }; + } + + @Override + public SimpleVersionedSerializer getCommittableSerializer() { + return new SimpleVersionedSerializer<>() { + @Override + public int getVersion() { + return 0; + } + + @Override + public byte[] serialize(TaskResult taskResult) { + return new byte[0]; + } + + @Override + public TaskResult deserialize(int i, byte[] bytes) { + return null; + } + }; + } + + @Override + public void addPostCommitTopology(DataStream> committables) { + committables + .flatMap( + new FlatMapFunction, TaskResult>() { + @Override + public void flatMap( + CommittableMessage taskResultCommittableMessage, + Collector collector) { + if (taskResultCommittableMessage instanceof CommittableWithLineage) { + collector.collect( + ((CommittableWithLineage) taskResultCommittableMessage) + .getCommittable()); + } + } + }) + .transform( + DUMMY_NAME, + TypeInformation.of(Void.class), + new LockRemover(new TestingLockFactory(), Lists.newArrayList(TASKS[0]))); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java index 876d64214560..3aee05322561 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.flink.maintenance.operator; -import static org.apache.iceberg.flink.maintenance.operator.FlinkStreamingTestUtils.closeJobClient; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -35,6 +34,7 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.hadoop.fs.Path; +import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; @@ -161,7 +161,12 @@ void testSource() throws Exception { } // The first non-empty event should contain the expected value - return newEvent.equals(new TableChange(1, 0, size, 0L, 1)); + return newEvent.equals( + TableChange.builder() + .dataFileCount(1) + .dataFileSizeInBytes(size) + .commitCount(1) + .build()); }); } finally { closeJobClient(jobClient); @@ -297,17 +302,17 @@ void testMaxReadBack() { new MonitorSource.TableChangeIterator(tableLoader, null, 1); // For a single maxReadBack we only get a single change - assertThat(iterator.next().commitNum()).isEqualTo(1); + assertThat(iterator.next().commitCount()).isEqualTo(1); iterator = new MonitorSource.TableChangeIterator(tableLoader, null, 2); // Expecting 2 commits/snapshots for maxReadBack=2 - assertThat(iterator.next().commitNum()).isEqualTo(2); + assertThat(iterator.next().commitCount()).isEqualTo(2); iterator = new MonitorSource.TableChangeIterator(tableLoader, null, Long.MAX_VALUE); // For maxReadBack Long.MAX_VALUE we get every change - assertThat(iterator.next().commitNum()).isEqualTo(3); + assertThat(iterator.next().commitCount()).isEqualTo(3); } @Test @@ -322,7 +327,7 @@ void testSkipReplace() { new MonitorSource.TableChangeIterator(tableLoader, null, Long.MAX_VALUE); // Read the current snapshot - assertThat(iterator.next().commitNum()).isEqualTo(1); + assertThat(iterator.next().commitCount()).isEqualTo(1); // Create a DataOperations.REPLACE snapshot Table table = tableLoader.loadTable(); @@ -348,15 +353,19 @@ private static TableChange tableChangeWithLastSnapshot(Table table, TableChange List deleteFiles = Lists.newArrayList(table.currentSnapshot().addedDeleteFiles(table.io()).iterator()); - long dataSize = dataFiles.stream().mapToLong(d -> d.fileSizeInBytes()).sum(); - long deleteSize = deleteFiles.stream().mapToLong(d -> d.fileSizeInBytes()).sum(); - boolean hasDelete = table.currentSnapshot().addedDeleteFiles(table.io()).iterator().hasNext(); - - return new TableChange( - previous.dataFileNum() + dataFiles.size(), - previous.deleteFileNum() + deleteFiles.size(), - previous.dataFileSize() + dataSize, - previous.deleteFileSize() + deleteSize, - previous.commitNum() + 1); + long dataSize = dataFiles.stream().mapToLong(ContentFile::fileSizeInBytes).sum(); + long deleteRecordCount = deleteFiles.stream().mapToLong(DeleteFile::recordCount).sum(); + + TableChange newChange = previous.copy(); + newChange.merge( + TableChange.builder() + .dataFileCount(dataFiles.size()) + .dataFileSizeInBytes(dataSize) + // Currently we only test with equality deletes + .eqDeleteFileCount(deleteFiles.size()) + .eqDeleteRecordCount(deleteRecordCount) + .commitCount(1) + .build()); + return newChange; } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java new file mode 100644 index 000000000000..fba4a12d9c6b --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java @@ -0,0 +1,686 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import static org.apache.iceberg.flink.maintenance.operator.ConstantsForTests.DUMMY_NAME; +import static org.apache.iceberg.flink.maintenance.operator.ConstantsForTests.EVENT_TIME; +import static org.apache.iceberg.flink.maintenance.operator.ConstantsForTests.EVENT_TIME_2; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.CONCURRENT_RUN_THROTTLED; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.GROUP_VALUE_DEFAULT; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.NOTHING_TO_TRIGGER; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.RATE_LIMITER_TRIGGERED; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.TRIGGERED; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.time.Duration; +import java.util.Iterator; +import java.util.List; +import java.util.stream.Stream; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.operators.KeyedProcessOperator; +import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +class TestTriggerManager extends OperatorTestBase { + private static final long DELAY = 10L; + private static final String NAME_1 = "name1"; + private static final String NAME_2 = "name2"; + private long processingTime = 0L; + private TriggerLockFactory lockFactory; + private TriggerLockFactory.Lock lock; + private TriggerLockFactory.Lock recoveringLock; + + @BeforeEach + void before() { + sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); + this.lockFactory = lockFactory(); + lockFactory.open(); + this.lock = lockFactory.createLock(); + this.recoveringLock = lockFactory.createRecoveryLock(); + lock.unlock(); + recoveringLock.unlock(); + MetricsReporterFactoryForTests.reset(); + } + + @AfterEach + void after() throws IOException { + lockFactory.close(); + } + + @Test + void testCommitCount() throws Exception { + TriggerManager manager = + manager(sql.tableLoader(TABLE_NAME), new TriggerEvaluator.Builder().commitCount(3).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(1).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(3).build(), 2); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(10).build(), 3); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(1).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(1).build(), 3); + + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(1).build(), 4); + } + } + + @Test + void testDataFileCount() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), new TriggerEvaluator.Builder().dataFileCount(3).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(1).build(), 0); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(3).build(), 2); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(5).build(), 3); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(1).build(), 3); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(2).build(), 4); + } + } + + @Test + void testDataFileSizeInBytes() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().dataFileSizeInBytes(3).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSizeInBytes(1L).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSizeInBytes(2L).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSizeInBytes(5L).build(), 2); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSizeInBytes(1L).build(), 2); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSizeInBytes(2L).build(), 3); + } + } + + @Test + void testPosDeleteFileCount() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().posDeleteFileCount(3).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(1).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(3).build(), 2); + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(10).build(), 3); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(1).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(1).build(), 3); + + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(1).build(), 4); + } + } + + @Test + void testPosDeleteRecordCount() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().posDeleteRecordCount(3).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult( + testHarness, TableChange.builder().posDeleteRecordCount(1L).build(), 0); + addEventAndCheckResult( + testHarness, TableChange.builder().posDeleteRecordCount(2L).build(), 1); + addEventAndCheckResult( + testHarness, TableChange.builder().posDeleteRecordCount(5L).build(), 2); + + // No trigger in this case + addEventAndCheckResult( + testHarness, TableChange.builder().posDeleteRecordCount(1L).build(), 2); + + addEventAndCheckResult( + testHarness, TableChange.builder().posDeleteRecordCount(2L).build(), 3); + } + } + + @Test + void testEqDeleteFileCount() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().eqDeleteFileCount(3).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(1).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(3).build(), 2); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(10).build(), 3); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(1).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(1).build(), 3); + + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(1).build(), 4); + } + } + + @Test + void testEqDeleteRecordCount() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().eqDeleteRecordCount(3).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteRecordCount(1L).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteRecordCount(2L).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteRecordCount(5L).build(), 2); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteRecordCount(1L).build(), 2); + + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteRecordCount(2L).build(), 3); + } + } + + @Test + void testTimeout() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().timeout(Duration.ofSeconds(1)).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + TableChange event = TableChange.builder().dataFileCount(1).commitCount(1).build(); + + // Wait for some time + testHarness.processElement(event, EVENT_TIME); + assertThat(testHarness.extractOutputValues()).isEmpty(); + + // Wait for the timeout to expire + long newTime = EVENT_TIME + Duration.ofSeconds(1).toMillis(); + testHarness.setProcessingTime(newTime); + testHarness.processElement(event, newTime); + assertThat(testHarness.extractOutputValues()).hasSize(1); + + // Remove the lock to allow the next trigger + lock.unlock(); + + // Send a new event + testHarness.setProcessingTime(newTime + 1); + testHarness.processElement(event, newTime); + + // No trigger yet + assertThat(testHarness.extractOutputValues()).hasSize(1); + + // Send a new event + newTime += Duration.ofSeconds(1).toMillis(); + testHarness.setProcessingTime(newTime); + testHarness.processElement(event, newTime); + + // New trigger should arrive + assertThat(testHarness.extractOutputValues()).hasSize(2); + } + } + + @Test + void testStateRestore() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TriggerManager manager = manager(tableLoader); + OperatorSubtaskState state; + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + testHarness.processElement( + TableChange.builder().dataFileCount(1).commitCount(1).build(), EVENT_TIME); + + assertThat(testHarness.extractOutputValues()).isEmpty(); + + state = testHarness.snapshot(1, EVENT_TIME); + } + + // Restore the state, write some more data, create a checkpoint, check the data which is written + manager = manager(tableLoader); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.initializeState(state); + testHarness.open(); + + // Arrives the first real change which triggers the recovery process + testHarness.processElement(TableChange.builder().commitCount(1).build(), EVENT_TIME_2); + assertTriggers( + testHarness.extractOutputValues(), + Lists.newArrayList(Trigger.recovery(testHarness.getProcessingTime()))); + + // Remove the lock to allow the next trigger + recoveringLock.unlock(); + testHarness.setProcessingTime(EVENT_TIME_2); + // At this point the output contains the recovery trigger and the real trigger + assertThat(testHarness.extractOutputValues()).hasSize(2); + } + } + + @Test + void testMinFireDelay() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TriggerManager manager = manager(tableLoader, DELAY, 1); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(2).build(), 1); + long currentTime = testHarness.getProcessingTime(); + + // No new fire yet + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(2).build(), 1); + + // Check that the trigger fired after the delay + testHarness.setProcessingTime(currentTime + DELAY); + assertThat(testHarness.extractOutputValues()).hasSize(2); + } + } + + @Test + void testLockCheckDelay() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TriggerManager manager = manager(tableLoader, 1, DELAY); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(2).build(), 1); + + // Create a lock to prevent execution, and check that there is no result + assertThat(lock.tryLock()).isTrue(); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(2).build(), 1); + long currentTime = testHarness.getProcessingTime(); + + // Remove the lock, and still no trigger + lock.unlock(); + assertThat(testHarness.extractOutputValues()).hasSize(1); + + // Check that the trigger fired after the delay + testHarness.setProcessingTime(currentTime + DELAY); + assertThat(testHarness.extractOutputValues()).hasSize(2); + } + } + + /** + * Simulating recovery scenarios where there is a leftover table lock, and ongoing maintenance + * task. + * + * @param locked if a lock exists on the table on job recovery + * @param runningTask is running and continues to run after job recovery + */ + @ParameterizedTest + @MethodSource("parametersForTestRecovery") + void testRecovery(boolean locked, boolean runningTask) throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TriggerManager manager = manager(tableLoader); + OperatorSubtaskState state; + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + state = testHarness.snapshot(1, EVENT_TIME); + } + + if (locked) { + assertThat(lock.tryLock()).isTrue(); + } + + manager = manager(tableLoader); + List expected = Lists.newArrayListWithExpectedSize(3); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.initializeState(state); + testHarness.open(); + + ++processingTime; + expected.add(Trigger.recovery(processingTime)); + testHarness.setProcessingTime(processingTime); + testHarness.processElement(TableChange.builder().commitCount(2).build(), processingTime); + assertTriggers(testHarness.extractOutputValues(), expected); + + // Nothing happens until the recovery is finished + ++processingTime; + testHarness.setProcessingTime(processingTime); + assertTriggers(testHarness.extractOutputValues(), expected); + + if (runningTask) { + // Simulate the action of the recovered maintenance task lock removal when it finishes + lock.unlock(); + } + + // Still no results as the recovery is ongoing + ++processingTime; + testHarness.setProcessingTime(processingTime); + testHarness.processElement(TableChange.builder().commitCount(2).build(), processingTime); + assertTriggers(testHarness.extractOutputValues(), expected); + + // Simulate the action of removing lock and recoveryLock by downstream lock cleaner when it + // received recovery trigger + lock.unlock(); + recoveringLock.unlock(); + + // Emit only a single trigger + ++processingTime; + testHarness.setProcessingTime(processingTime); + // Releasing lock will create a new snapshot, and we receive this in the trigger + expected.add( + Trigger.create( + processingTime, + (SerializableTable) SerializableTable.copyOf(tableLoader.loadTable()), + 0)); + assertTriggers(testHarness.extractOutputValues(), expected); + } + } + + @Test + void testTriggerMetrics() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource source = + new ManualSource<>(env, TypeInformation.of(TableChange.class)); + CollectingSink sink = new CollectingSink<>(); + + TriggerManager manager = + new TriggerManager( + tableLoader, + lockFactory, + Lists.newArrayList(NAME_1, NAME_2), + Lists.newArrayList( + new TriggerEvaluator.Builder().commitCount(2).build(), + new TriggerEvaluator.Builder().commitCount(4).build()), + 1L, + 1L); + source + .dataStream() + .keyBy(unused -> true) + .process(manager) + .name(DUMMY_NAME) + .forceNonParallel() + .sinkTo(sink); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + // This one doesn't trigger - tests NOTHING_TO_TRIGGER + source.sendRecord(TableChange.builder().commitCount(1).build()); + + Awaitility.await() + .until( + () -> { + Long notingCounter = + MetricsReporterFactoryForTests.counter( + DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + NOTHING_TO_TRIGGER); + return notingCounter != null && notingCounter.equals(1L); + }); + + // Trigger one of the tasks - tests TRIGGERED + source.sendRecord(TableChange.builder().commitCount(1).build()); + // Wait until we receive the trigger + assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); + assertThat( + MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED)) + .isEqualTo(1L); + lock.unlock(); + + // Trigger both of the tasks - tests TRIGGERED + source.sendRecord(TableChange.builder().commitCount(2).build()); + // Wait until we receive the trigger + assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); + lock.unlock(); + assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); + lock.unlock(); + assertThat( + MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED)) + .isEqualTo(2L); + assertThat( + MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + NAME_2 + "." + TRIGGERED)) + .isEqualTo(1L); + + // Final check all the counters + MetricsReporterFactoryForTests.assertCounters( + new ImmutableMap.Builder() + .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + RATE_LIMITER_TRIGGERED, -1L) + .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + CONCURRENT_RUN_THROTTLED, -1L) + .put(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED, 2L) + .put(DUMMY_NAME + "." + NAME_2 + "." + TRIGGERED, 1L) + .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + NOTHING_TO_TRIGGER, 1L) + .build()); + } finally { + closeJobClient(jobClient); + } + } + + @Test + void testRateLimiterMetrics() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource source = + new ManualSource<>(env, TypeInformation.of(TableChange.class)); + CollectingSink sink = new CollectingSink<>(); + + // High delay, so only triggered once + TriggerManager manager = manager(tableLoader, 1_000_000L, 1L); + source + .dataStream() + .keyBy(unused -> true) + .process(manager) + .name(DUMMY_NAME) + .forceNonParallel() + .sinkTo(sink); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + // Start the first trigger + source.sendRecord(TableChange.builder().commitCount(2).build()); + assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); + + // Remove the lock to allow the next trigger + lock.unlock(); + + // The second trigger will be blocked + source.sendRecord(TableChange.builder().commitCount(2).build()); + Awaitility.await() + .until( + () -> + MetricsReporterFactoryForTests.counter( + DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + RATE_LIMITER_TRIGGERED) + .equals(1L)); + + // Final check all the counters + assertCounters(1L, 0L); + } finally { + closeJobClient(jobClient); + } + } + + @Test + void testConcurrentRunMetrics() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource source = + new ManualSource<>(env, TypeInformation.of(TableChange.class)); + CollectingSink sink = new CollectingSink<>(); + + // High delay, so only triggered once + TriggerManager manager = manager(tableLoader, 1L, 1_000_000L); + source + .dataStream() + .keyBy(unused -> true) + .process(manager) + .name(DUMMY_NAME) + .forceNonParallel() + .sinkTo(sink); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + // Start the first trigger - notice that we do not remove the lock after the trigger + source.sendRecord(TableChange.builder().commitCount(2).build()); + assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); + + // The second trigger will be blocked by the lock + source.sendRecord(TableChange.builder().commitCount(2).build()); + Awaitility.await() + .until( + () -> + MetricsReporterFactoryForTests.counter( + DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + CONCURRENT_RUN_THROTTLED) + .equals(1L)); + + // Final check all the counters + assertCounters(0L, 1L); + } finally { + closeJobClient(jobClient); + } + } + + private static Stream parametersForTestRecovery() { + return Stream.of( + Arguments.of(true, false), + Arguments.of(true, false), + Arguments.of(false, true), + Arguments.of(false, false)); + } + + private void assertCounters(long rateLimiterTrigger, long concurrentRunTrigger) { + MetricsReporterFactoryForTests.assertCounters( + new ImmutableMap.Builder() + .put( + DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + RATE_LIMITER_TRIGGERED, + rateLimiterTrigger) + .put( + DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + CONCURRENT_RUN_THROTTLED, + concurrentRunTrigger) + .put(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED, 1L) + .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + NOTHING_TO_TRIGGER, 0L) + .build()); + } + + private KeyedOneInputStreamOperatorTestHarness harness( + TriggerManager manager) throws Exception { + return new KeyedOneInputStreamOperatorTestHarness<>( + new KeyedProcessOperator<>(manager), value -> true, Types.BOOLEAN); + } + + private void addEventAndCheckResult( + OneInputStreamOperatorTestHarness testHarness, + TableChange event, + int expectedSize) + throws Exception { + ++processingTime; + testHarness.setProcessingTime(processingTime); + testHarness.processElement(event, processingTime); + assertThat(testHarness.extractOutputValues()).hasSize(expectedSize); + // Remove the lock to allow the next trigger + lock.unlock(); + } + + private TriggerManager manager(TableLoader tableLoader, TriggerEvaluator evaluator) { + return new TriggerManager( + tableLoader, lockFactory, Lists.newArrayList(NAME_1), Lists.newArrayList(evaluator), 1, 1); + } + + private TriggerManager manager( + TableLoader tableLoader, long minFireDelayMs, long lockCheckDelayMs) { + return new TriggerManager( + tableLoader, + lockFactory, + Lists.newArrayList(NAME_1), + Lists.newArrayList(new TriggerEvaluator.Builder().commitCount(2).build()), + minFireDelayMs, + lockCheckDelayMs); + } + + private TriggerManager manager(TableLoader tableLoader) { + return manager(tableLoader, new TriggerEvaluator.Builder().commitCount(2).build()); + } + + private static void assertTriggers(List expected, List actual) { + assertThat(actual).hasSize(expected.size()); + for (int i = 0; i < expected.size(); ++i) { + Trigger expectedTrigger = expected.get(i); + Trigger actualTrigger = actual.get(i); + assertThat(actualTrigger.timestamp()).isEqualTo(expectedTrigger.timestamp()); + assertThat(actualTrigger.taskId()).isEqualTo(expectedTrigger.taskId()); + assertThat(actualTrigger.isRecovery()).isEqualTo(expectedTrigger.isRecovery()); + if (expectedTrigger.table() == null) { + assertThat(actualTrigger.table()).isNull(); + } else { + Iterator expectedSnapshots = expectedTrigger.table().snapshots().iterator(); + Iterator actualSnapshots = actualTrigger.table().snapshots().iterator(); + while (expectedSnapshots.hasNext()) { + assertThat(actualSnapshots.hasNext()).isTrue(); + assertThat(expectedSnapshots.next().snapshotId()) + .isEqualTo(actualSnapshots.next().snapshotId()); + } + } + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/SinkTestUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/SinkTestUtil.java new file mode 100644 index 000000000000..1cf55bcdc817 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/SinkTestUtil.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Collection; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableSummary; +import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; +import org.apache.flink.streaming.runtime.streamrecord.StreamElement; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + +class SinkTestUtil { + + private SinkTestUtil() {} + + @SuppressWarnings("unchecked") + static List transformsToStreamElement(Collection elements) { + return elements.stream() + .map( + element -> { + if (element instanceof StreamRecord) { + return new StreamRecord<>( + ((StreamRecord>) element).getValue()); + } + return (StreamElement) element; + }) + .collect(Collectors.toList()); + } + + static CommittableSummary extractAndAssertCommittableSummary(StreamElement element) { + final Object value = element.asRecord().getValue(); + assertThat(value).isInstanceOf(CommittableSummary.class); + return (CommittableSummary) value; + } + + static CommittableWithLineage extractAndAssertCommittableWithLineage( + StreamElement element) { + final Object value = element.asRecord().getValue(); + assertThat(value).isInstanceOf(CommittableWithLineage.class); + return (CommittableWithLineage) value; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java index 8faae1b05a4e..3299e7a97776 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java @@ -40,7 +40,6 @@ import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.io.BaseTaskWriter; import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -210,8 +209,10 @@ public void testCompressionOrc() throws Exception { .containsEntry(TableProperties.ORC_COMPRESSION_STRATEGY, "speed"); } - private static OneInputStreamOperatorTestHarness createIcebergStreamWriter( - Table icebergTable, TableSchema flinkSchema, Map override) throws Exception { + private static OneInputStreamOperatorTestHarness + createIcebergStreamWriter( + Table icebergTable, TableSchema flinkSchema, Map override) + throws Exception { RowType flinkRowType = FlinkSink.toFlinkRowType(icebergTable.schema(), flinkSchema); FlinkWriteConf flinkWriteConfig = new FlinkWriteConf( @@ -219,7 +220,7 @@ private static OneInputStreamOperatorTestHarness createIce IcebergStreamWriter streamWriter = FlinkSink.createStreamWriter(() -> icebergTable, flinkWriteConfig, flinkRowType, null); - OneInputStreamOperatorTestHarness harness = + OneInputStreamOperatorTestHarness harness = new OneInputStreamOperatorTestHarness<>(streamWriter, 1, 1, 0); harness.setup(); @@ -230,7 +231,7 @@ private static OneInputStreamOperatorTestHarness createIce private static Map appenderProperties( Table table, TableSchema schema, Map override) throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter(table, schema, override)) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java index 75e397d3f203..df8c3c79d3e3 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java @@ -20,28 +20,37 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.IOException; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; import org.apache.iceberg.DistributionMode; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Parameter; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; import org.apache.iceberg.TableProperties; import org.apache.iceberg.flink.FlinkWriteOptions; import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.sink.shuffle.StatisticsType; +import org.apache.iceberg.flink.source.BoundedTestSource; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Types; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -177,4 +186,309 @@ public void testOverrideWriteConfigWithUnknownDistributionMode() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid distribution mode: UNRECOGNIZED"); } + + @TestTemplate + public void testRangeDistributionWithoutSortOrderUnpartitioned() throws Exception { + assumeThat(partitioned).isFalse(); + + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + + int numOfCheckpoints = 6; + DataStream dataStream = + env.addSource( + createRangeDistributionBoundedSource(createCharRows(numOfCheckpoints, 10)), + ROW_TYPE_INFO); + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism); + + // Range distribution requires either sort order or partition spec defined + assertThatThrownBy(builder::append) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Invalid write distribution mode: range. Need to define sort order or partition spec."); + } + + @TestTemplate + public void testRangeDistributionWithoutSortOrderPartitioned() throws Exception { + assumeThat(partitioned).isTrue(); + + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + + int numOfCheckpoints = 6; + DataStream dataStream = + env.addSource( + createRangeDistributionBoundedSource(createCharRows(numOfCheckpoints, 10)), + ROW_TYPE_INFO); + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism); + + // sort based on partition columns + builder.append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(numOfCheckpoints); + } + + @TestTemplate + public void testRangeDistributionWithSortOrder() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + table.replaceSortOrder().asc("data").commit(); + + int numOfCheckpoints = 6; + DataStream dataStream = + env.addSource( + createRangeDistributionBoundedSource(createCharRows(numOfCheckpoints, 10)), + ROW_TYPE_INFO); + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .rangeDistributionStatisticsType(StatisticsType.Map) + .append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(numOfCheckpoints); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + if (partitioned) { + for (Snapshot snapshot : rangePartitionedCycles) { + List addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // up to 26 partitions + assertThat(addedDataFiles).hasSizeLessThanOrEqualTo(26); + } + } else { + for (Snapshot snapshot : rangePartitionedCycles) { + List addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // each writer task should only write one file for non-partition sort column + assertThat(addedDataFiles).hasSize(parallelism); + // verify there is no overlap in min-max stats range + if (parallelism == 2) { + assertIdColumnStatsNoRangeOverlap(addedDataFiles.get(0), addedDataFiles.get(1)); + } + } + } + } + + @TestTemplate + public void testRangeDistributionSketchWithSortOrder() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + table.replaceSortOrder().asc("id").commit(); + + int numOfCheckpoints = 6; + DataStream dataStream = + env.addSource( + createRangeDistributionBoundedSource(createIntRows(numOfCheckpoints, 1_000)), + ROW_TYPE_INFO); + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .rangeDistributionStatisticsType(StatisticsType.Sketch) + .append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(numOfCheckpoints); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + // since the input has a single value for the data column, + // it is always the same partition. Hence there is no difference + // for partitioned or not + for (Snapshot snapshot : rangePartitionedCycles) { + List addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // each writer task should only write one file for non-partition sort column + assertThat(addedDataFiles).hasSize(parallelism); + // verify there is no overlap in min-max stats range + if (parallelism == 2) { + assertIdColumnStatsNoRangeOverlap(addedDataFiles.get(0), addedDataFiles.get(1)); + } + } + } + + /** Test migration from Map stats to Sketch stats */ + @TestTemplate + public void testRangeDistributionStatisticsMigration() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + table.replaceSortOrder().asc("id").commit(); + + int numOfCheckpoints = 4; + List> rowsPerCheckpoint = Lists.newArrayListWithCapacity(numOfCheckpoints); + for (int checkpointId = 0; checkpointId < numOfCheckpoints; ++checkpointId) { + // checkpointId 2 would emit 11_000 records which is larger than + // the OPERATOR_SKETCH_SWITCH_THRESHOLD of 10_000. + // This should trigger the stats migration. + int maxId = checkpointId < 1 ? 1_000 : 11_000; + List rows = Lists.newArrayListWithCapacity(maxId); + for (int j = 0; j < maxId; ++j) { + // fixed value "a" for the data (possible partition column) + rows.add(Row.of(j, "a")); + } + + rowsPerCheckpoint.add(rows); + } + + DataStream dataStream = + env.addSource(createRangeDistributionBoundedSource(rowsPerCheckpoint), ROW_TYPE_INFO); + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .rangeDistributionStatisticsType(StatisticsType.Auto) + .append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(numOfCheckpoints); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + // since the input has a single value for the data column, + // it is always the same partition. Hence there is no difference + // for partitioned or not + for (Snapshot snapshot : rangePartitionedCycles) { + List addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // each writer task should only write one file for non-partition sort column + // sometimes + assertThat(addedDataFiles).hasSize(parallelism); + // verify there is no overlap in min-max stats range + if (parallelism == 2) { + assertIdColumnStatsNoRangeOverlap(addedDataFiles.get(0), addedDataFiles.get(1)); + } + } + } + + private BoundedTestSource createRangeDistributionBoundedSource( + List> rowsPerCheckpoint) { + return new BoundedTestSource<>(rowsPerCheckpoint); + } + + private List> createCharRows(int numOfCheckpoints, int countPerChar) { + List> rowsPerCheckpoint = Lists.newArrayListWithCapacity(numOfCheckpoints); + for (int checkpointId = 0; checkpointId < numOfCheckpoints; ++checkpointId) { + List rows = Lists.newArrayListWithCapacity(26 * countPerChar); + for (int j = 0; j < countPerChar; ++j) { + for (char c = 'a'; c <= 'z'; ++c) { + rows.add(Row.of(1, String.valueOf(c))); + } + } + + rowsPerCheckpoint.add(rows); + } + + return rowsPerCheckpoint; + } + + private List> createIntRows(int numOfCheckpoints, int maxId) { + List> rowsPerCheckpoint = Lists.newArrayListWithCapacity(numOfCheckpoints); + for (int checkpointId = 0; checkpointId < numOfCheckpoints; ++checkpointId) { + List rows = Lists.newArrayListWithCapacity(maxId); + for (int j = 0; j < maxId; ++j) { + // fixed value "a" for the data (possible partition column) + rows.add(Row.of(j, "a")); + } + + rowsPerCheckpoint.add(rows); + } + + return rowsPerCheckpoint; + } + + private void assertIdColumnStatsNoRangeOverlap(DataFile file1, DataFile file2) { + // id column has fieldId 1 + int file1LowerBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file1.lowerBounds().get(1)); + int file1UpperBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file1.upperBounds().get(1)); + int file2LowerBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file2.lowerBounds().get(1)); + int file2UpperBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file2.upperBounds().get(1)); + + if (file1LowerBound < file2LowerBound) { + assertThat(file1UpperBound).isLessThanOrEqualTo(file2LowerBound); + } else { + assertThat(file2UpperBound).isLessThanOrEqualTo(file1LowerBound); + } + } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java new file mode 100644 index 000000000000..a5799288b5e3 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java @@ -0,0 +1,255 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.apache.iceberg.expressions.Expressions.bucket; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.connector.datagen.source.DataGeneratorSource; +import org.apache.flink.connector.datagen.source.GeneratorFunction; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.RegisterExtension; + +/** + * Test range distribution with bucketing partition column. Compared to hash distribution, range + * distribution is more general to handle bucketing column while achieving even distribution of + * traffic to writer tasks. + * + *
    + *
  • keyBy on low cardinality (e.g. + * 60) may not achieve balanced data distribution. + *
  • number of buckets (e.g. 60) is not divisible by the writer parallelism (e.g. 40). + *
  • number of buckets (e.g. 60) is smaller than the writer parallelism (e.g. 120). + *
+ */ +@Timeout(value = 30) +@Disabled // https://github.com/apache/iceberg/pull/11305#issuecomment-2415207097 +public class TestFlinkIcebergSinkRangeDistributionBucketing { + private static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = + new Configuration() + // disable classloader check as Avro may cache class/object in the serializers. + .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); + + // max supported parallelism is 16 (= 4 x 4) + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(4) + .setNumberSlotsPerTaskManager(4) + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) + .build()); + + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + private static final int NUM_BUCKETS = 4; + private static final int NUM_OF_CHECKPOINTS = 6; + private static final int ROW_COUNT_PER_CHECKPOINT = 200; + private static final Schema SCHEMA = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "uuid", Types.UUIDType.get()), + Types.NestedField.optional(3, "data", Types.StringType.get())); + private static final PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA).hour("ts").bucket("uuid", NUM_BUCKETS).build(); + private static final RowType ROW_TYPE = FlinkSchemaUtil.convert(SCHEMA); + + private TableLoader tableLoader; + private Table table; + + @BeforeEach + public void before() throws IOException { + this.tableLoader = CATALOG_EXTENSION.tableLoader(); + this.table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SCHEMA, + SPEC, + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, FileFormat.PARQUET.name())); + + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + + // Assuming ts is on ingestion/processing time. Writer only writes to 1 or 2 hours concurrently. + // Only sort on the bucket column to avoid each writer task writes to 60 buckets/files + // concurrently. + table.replaceSortOrder().asc(bucket("uuid", NUM_BUCKETS)).commit(); + } + + @AfterEach + public void after() throws Exception { + CATALOG_EXTENSION.catalog().dropTable(TestFixtures.TABLE_IDENTIFIER); + } + + /** number of buckets 4 matches writer parallelism of 4 */ + @Test + public void testBucketNumberEqualsToWriterParallelism() throws Exception { + testParallelism(4); + } + + /** number of buckets 4 is less than writer parallelism of 6 */ + @Test + public void testBucketNumberLessThanWriterParallelismNotDivisible() throws Exception { + testParallelism(6); + } + + /** number of buckets 4 is less than writer parallelism of 8 */ + @Test + public void testBucketNumberLessThanWriterParallelismDivisible() throws Exception { + testParallelism(8); + } + + /** number of buckets 4 is greater than writer parallelism of 3 */ + @Test + public void testBucketNumberHigherThanWriterParallelismNotDivisible() throws Exception { + testParallelism(3); + } + + /** number of buckets 4 is greater than writer parallelism of 2 */ + @Test + public void testBucketNumberHigherThanWriterParallelismDivisible() throws Exception { + testParallelism(2); + } + + private void testParallelism(int parallelism) throws Exception { + try (StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism)) { + + DataGeneratorSource generatorSource = + new DataGeneratorSource<>( + new RowGenerator(), + ROW_COUNT_PER_CHECKPOINT * NUM_OF_CHECKPOINTS, + RateLimiterStrategy.perCheckpoint(ROW_COUNT_PER_CHECKPOINT), + FlinkCompatibilityUtil.toTypeInfo(ROW_TYPE)); + DataStream dataStream = + env.fromSource(generatorSource, WatermarkStrategy.noWatermarks(), "Data Generator"); + + FlinkSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the oldest snapshot to the newest snapshot + List snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Source rate limit per checkpoint cycle may not be super precise. + // There could be more checkpoint cycles and commits than planned. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(NUM_OF_CHECKPOINTS); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + for (Snapshot snapshot : rangePartitionedCycles) { + List addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + assertThat(addedDataFiles) + .hasSizeLessThanOrEqualTo(maxAddedDataFilesPerCheckpoint(parallelism)); + } + } + } + + /** + * Traffic is not perfectly balanced across all buckets in the small sample size Range + * distribution of the bucket id may cross subtask boundary. Hence the number of committed data + * files per checkpoint maybe larger than writer parallelism or the number of buckets. But it + * should not be more than the sum of those two. Without range distribution, the number of data + * files per commit can be 4x of parallelism (as the number of buckets is 4). + */ + private int maxAddedDataFilesPerCheckpoint(int parallelism) { + return NUM_BUCKETS + parallelism; + } + + private static class RowGenerator implements GeneratorFunction { + // use constant timestamp so that all rows go to the same hourly partition + private final long ts = System.currentTimeMillis(); + + @Override + public RowData map(Long index) throws Exception { + // random uuid should result in relatively balanced distribution across buckets + UUID uuid = UUID.randomUUID(); + ByteBuffer uuidByteBuffer = ByteBuffer.allocate(16); + uuidByteBuffer.putLong(uuid.getMostSignificantBits()); + uuidByteBuffer.putLong(uuid.getLeastSignificantBits()); + return GenericRowData.of( + TimestampData.fromEpochMillis(ts), + uuidByteBuffer.array(), + StringData.fromString("row-" + index)); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java index 577c54976b9a..b283b8390a2b 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -30,6 +30,7 @@ import org.apache.flink.types.Row; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.DistributionMode; import org.apache.iceberg.FileFormat; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.ParameterizedTestExtension; @@ -184,11 +185,21 @@ public void testUpsertModeCheck() throws Exception { .hasMessage( "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); - assertThatThrownBy( - () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) - .isInstanceOf(IllegalStateException.class) - .hasMessage( - "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + if (writeDistributionMode.equals(DistributionMode.RANGE.modeName()) && !partitioned) { + // validation error thrown from distributeDataStream + assertThatThrownBy( + () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Invalid write distribution mode: range. Need to define sort order or partition spec."); + } else { + // validation error thrown from appendWriter + assertThatThrownBy( + () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + } } @TestTemplate diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java index fc33c2fea5e6..da4ac2ada677 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java @@ -48,30 +48,30 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.StructLikeSet; -public class TestFlinkIcebergSinkV2Base { +class TestFlinkIcebergSinkV2Base { - protected static final int FORMAT_V2 = 2; - protected static final TypeInformation ROW_TYPE_INFO = + static final int FORMAT_V2 = 2; + static final TypeInformation ROW_TYPE_INFO = new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); - protected static final int ROW_ID_POS = 0; - protected static final int ROW_DATA_POS = 1; + static final int ROW_ID_POS = 0; + static final int ROW_DATA_POS = 1; - protected TableLoader tableLoader; - protected Table table; - protected StreamExecutionEnvironment env; + TableLoader tableLoader; + Table table; + StreamExecutionEnvironment env; @Parameter(index = 0) - protected FileFormat format; + FileFormat format; @Parameter(index = 1) - protected int parallelism = 1; + int parallelism = 1; @Parameter(index = 2) - protected boolean partitioned; + boolean partitioned; @Parameter(index = 3) - protected String writeDistributionMode; + String writeDistributionMode; @Parameters(name = "FileFormat={0}, Parallelism={1}, Partitioned={2}, WriteDistributionMode={3}") public static Object[][] parameters() { @@ -91,14 +91,14 @@ public static Object[][] parameters() { }; } - protected static final Map ROW_KIND_MAP = + static final Map ROW_KIND_MAP = ImmutableMap.of( "+I", RowKind.INSERT, "-D", RowKind.DELETE, "-U", RowKind.UPDATE_BEFORE, "+U", RowKind.UPDATE_AFTER); - protected Row row(String rowKind, int id, String data) { + Row row(String rowKind, int id, String data) { RowKind kind = ROW_KIND_MAP.get(rowKind); if (kind == null) { throw new IllegalArgumentException("Unknown row kind: " + rowKind); @@ -107,7 +107,7 @@ protected Row row(String rowKind, int id, String data) { return Row.ofKind(kind, id, data); } - protected void testUpsertOnIdDataKey(String branch) throws Exception { + void testUpsertOnIdDataKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of(row("+I", 1, "aaa"), row("+U", 1, "aaa"), row("+I", 2, "bbb")), @@ -128,7 +128,7 @@ protected void testUpsertOnIdDataKey(String branch) throws Exception { branch); } - protected void testChangeLogOnIdDataKey(String branch) throws Exception { + void testChangeLogOnIdDataKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of( @@ -157,7 +157,7 @@ protected void testChangeLogOnIdDataKey(String branch) throws Exception { branch); } - protected void testChangeLogOnSameKey(String branch) throws Exception { + void testChangeLogOnSameKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( // Checkpoint #1 @@ -185,7 +185,7 @@ protected void testChangeLogOnSameKey(String branch) throws Exception { branch); } - protected void testChangeLogOnDataKey(String branch) throws Exception { + void testChangeLogOnDataKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of( @@ -213,7 +213,7 @@ protected void testChangeLogOnDataKey(String branch) throws Exception { branch); } - protected void testUpsertOnDataKey(String branch) throws Exception { + void testUpsertOnDataKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of(row("+I", 1, "aaa"), row("+I", 2, "aaa"), row("+I", 3, "bbb")), @@ -235,7 +235,7 @@ protected void testUpsertOnDataKey(String branch) throws Exception { branch); } - protected void testChangeLogOnIdKey(String branch) throws Exception { + void testChangeLogOnIdKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of( @@ -285,7 +285,7 @@ protected void testChangeLogOnIdKey(String branch) throws Exception { } } - protected void testUpsertOnIdKey(String branch) throws Exception { + void testUpsertOnIdKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of(row("+I", 1, "aaa"), row("+U", 1, "bbb")), @@ -321,7 +321,7 @@ protected void testUpsertOnIdKey(String branch) throws Exception { } } - protected void testChangeLogs( + void testChangeLogs( List equalityFieldColumns, KeySelector keySelector, boolean insertAsUpsert, @@ -358,11 +358,11 @@ protected void testChangeLogs( } } - protected Record record(int id, String data) { + Record record(int id, String data) { return SimpleDataUtil.createRecord(id, data); } - private List findValidSnapshots() { + List findValidSnapshots() { List validSnapshots = Lists.newArrayList(); for (Snapshot snapshot : table.snapshots()) { if (snapshot.allManifests(table.io()).stream() @@ -373,11 +373,11 @@ private List findValidSnapshots() { return validSnapshots; } - private StructLikeSet expectedRowSet(Record... records) { + StructLikeSet expectedRowSet(Record... records) { return SimpleDataUtil.expectedRowSet(table, records); } - private StructLikeSet actualRowSet(long snapshotId, String... columns) throws IOException { + StructLikeSet actualRowSet(long snapshotId, String... columns) throws IOException { table.refresh(); StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); try (CloseableIterable reader = diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java index 0b0c55f51c32..56cba8f460e2 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java @@ -43,15 +43,24 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestFlinkIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Base { @RegisterExtension - private static final HadoopCatalogExtension CATALOG_EXTENSION = + static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); - @Parameter(index = 0) - private String branch; + @Parameter(index = 4) + protected String branch; - @Parameters(name = "branch = {0}") + @Parameters( + name = + "FileFormat={0}, Parallelism={1}, Partitioned={2}, WriteDistributionMode={3}, Branch={4}") public static Object[][] parameters() { - return new Object[][] {new Object[] {"main"}, new Object[] {"testBranch"}}; + return new Object[][] { + new Object[] { + FileFormat.AVRO, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE, "main" + }, + new Object[] { + FileFormat.AVRO, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE, "testBranch" + } + }; } @BeforeEach diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java index 53b7c4c0cc91..c21c3d5cc21b 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.flink.sink; -import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; import static org.assertj.core.api.Assertions.assertThat; import java.io.File; @@ -134,7 +133,9 @@ public void testUserProvidedManifestLocation() throws IOException { String operatorId = newOperatorUniqueId(); File userProvidedFolder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); Map props = - ImmutableMap.of(FLINK_MANIFEST_LOCATION, userProvidedFolder.getAbsolutePath() + "///"); + ImmutableMap.of( + ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION, + userProvidedFolder.getAbsolutePath() + "///"); ManifestOutputFileFactory factory = new ManifestOutputFileFactory(() -> table, props, flinkJobId, operatorId, 1, 1); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java new file mode 100644 index 000000000000..f11aae1d6923 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java @@ -0,0 +1,1445 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; +import static org.apache.iceberg.flink.sink.SinkTestUtil.extractAndAssertCommittableSummary; +import static org.apache.iceberg.flink.sink.SinkTestUtil.extractAndAssertCommittableWithLineage; +import static org.apache.iceberg.flink.sink.SinkTestUtil.transformsToStreamElement; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.TaskInfo; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessageSerializer; +import org.apache.flink.streaming.api.connector.sink2.CommittableSummary; +import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; +import org.apache.flink.streaming.api.connector.sink2.SinkV2Assertions; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.runtime.operators.sink.CommitterOperatorFactory; +import org.apache.flink.streaming.runtime.streamrecord.StreamElement; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.StreamTask; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.table.data.RowData; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.GenericManifestFile; +import org.apache.iceberg.ManifestContent; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestBase; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@ExtendWith(ParameterizedTestExtension.class) +class TestIcebergCommitter extends TestBase { + private static final Logger LOG = LoggerFactory.getLogger(TestIcebergCommitter.class); + public static final String OPERATOR_ID = "flink-sink"; + @TempDir File temporaryFolder; + + @TempDir File flinkManifestFolder; + + private Table table; + + private TableLoader tableLoader; + + @Parameter(index = 1) + private Boolean isStreamingMode; + + @Parameter(index = 2) + private String branch; + + private final String jobId = "jobId"; + private final long dataFIleRowCount = 5L; + + private final TestCommittableMessageTypeSerializer committableMessageTypeSerializer = + new TestCommittableMessageTypeSerializer(); + + private final DataFile dataFileTest1 = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-1.parquet") + .withFileSizeInBytes(0) + .withMetrics( + new Metrics( + dataFIleRowCount, + null, // no column sizes + ImmutableMap.of(1, 5L), // value count + ImmutableMap.of(1, 0L), // null count + null, + ImmutableMap.of(1, longToBuffer(0L)), // lower bounds + ImmutableMap.of(1, longToBuffer(4L)) // upper bounds + )) + .build(); + + private final DataFile dataFileTest2 = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-2.parquet") + .withFileSizeInBytes(0) + .withMetrics( + new Metrics( + dataFIleRowCount, + null, // no column sizes + ImmutableMap.of(1, 5L), // value count + ImmutableMap.of(1, 0L), // null count + null, + ImmutableMap.of(1, longToBuffer(0L)), // lower bounds + ImmutableMap.of(1, longToBuffer(4L)) // upper bounds + )) + .build(); + + @SuppressWarnings("checkstyle:NestedForDepth") + @Parameters(name = "formatVersion={0} isStreaming={1}, branch={2}") + protected static List parameters() { + List parameters = Lists.newArrayList(); + for (Boolean isStreamingMode : new Boolean[] {true, false}) { + for (int formatVersion : new int[] {1, 2}) { + parameters.add(new Object[] {formatVersion, isStreamingMode, SnapshotRef.MAIN_BRANCH}); + parameters.add(new Object[] {formatVersion, isStreamingMode, "test-branch"}); + } + } + return parameters; + } + + @BeforeEach + public void before() throws Exception { + String warehouse = temporaryFolder.getAbsolutePath(); + + String tablePath = warehouse.concat("/test"); + assertThat(new File(tablePath).mkdir()).as("Should create the table path correctly.").isTrue(); + + Map props = + ImmutableMap.of( + TableProperties.FORMAT_VERSION, + String.valueOf(formatVersion), + FLINK_MANIFEST_LOCATION, + flinkManifestFolder.getAbsolutePath(), + IcebergCommitter.MAX_CONTINUOUS_EMPTY_COMMITS, + "1"); + table = SimpleDataUtil.createTable(tablePath, props, false); + tableLoader = TableLoader.fromHadoopTable(tablePath); + } + + @TestTemplate + public void testCommitTxnWithoutDataFiles() throws Exception { + IcebergCommitter committer = getCommitter(); + SimpleDataUtil.assertTableRows(table, Lists.newArrayList(), branch); + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId, -1); + + for (long i = 1; i <= 3; i++) { + Committer.CommitRequest commitRequest = + buildCommitRequestFor(jobId, i, Lists.newArrayList()); + committer.commit(Lists.newArrayList(commitRequest)); + assertMaxCommittedCheckpointId(jobId, i); + assertSnapshotSize((int) i); + } + } + + @TestTemplate + public void testMxContinuousEmptyCommits() throws Exception { + table.updateProperties().set(IcebergCommitter.MAX_CONTINUOUS_EMPTY_COMMITS, "3").commit(); + IcebergCommitter committer = getCommitter(); + for (int i = 1; i <= 9; i++) { + Committer.CommitRequest commitRequest = + buildCommitRequestFor(jobId, i, Lists.newArrayList()); + committer.commit(Lists.newArrayList(commitRequest)); + assertFlinkManifests(0); + assertSnapshotSize(i / 3); + } + } + + @TestTemplate + public void testCommitTxn() throws Exception { + IcebergCommitter committer = getCommitter(); + assertSnapshotSize(0); + List rows = Lists.newArrayListWithExpectedSize(3); + for (int i = 1; i <= 3; i++) { + RowData rowData = SimpleDataUtil.createRowData(i, "hello" + i); + DataFile dataFile = writeDataFile("data-" + i, ImmutableList.of(rowData)); + rows.add(rowData); + WriteResult writeResult = of(dataFile); + Committer.CommitRequest commitRequest = + buildCommitRequestFor(jobId, i, Lists.newArrayList(writeResult)); + committer.commit(Lists.newArrayList(commitRequest)); + assertFlinkManifests(0); + SimpleDataUtil.assertTableRows(table, ImmutableList.copyOf(rows), branch); + assertSnapshotSize(i); + assertMaxCommittedCheckpointId(jobId, i); + Map summary = SimpleDataUtil.latestSnapshot(table, branch).summary(); + assertThat(summary) + .containsEntry("flink.test", "org.apache.iceberg.flink.sink.TestIcebergCommitter") + .containsEntry("added-data-files", "1") + .containsEntry("flink.operator-id", OPERATOR_ID) + .containsEntry("flink.job-id", "jobId"); + } + } + + @TestTemplate + public void testOrderedEventsBetweenCheckpoints() throws Exception { + // It's possible that two checkpoints happen in the following orders: + // 1. snapshotState for checkpoint#1; + // 2. snapshotState for checkpoint#2; + // 3. notifyCheckpointComplete for checkpoint#1; + // 4. notifyCheckpointComplete for checkpoint#2; + + long timestamp = 0; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + + harness.open(); + + assertMaxCommittedCheckpointId(jobId, -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello"); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + + processElement(jobId, 1, harness, 1, OPERATOR_ID, dataFile1); + assertMaxCommittedCheckpointId(jobId, -1L); + + // 1. snapshotState for checkpoint#1 + long firstCheckpointId = 1; + harness.snapshot(firstCheckpointId, ++timestamp); + assertFlinkManifests(1); + + RowData row2 = SimpleDataUtil.createRowData(2, "world"); + DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); + processElement(jobId, 2, harness, 1, OPERATOR_ID, dataFile2); + assertMaxCommittedCheckpointId(jobId, -1L); + + // 2. snapshotState for checkpoint#2 + long secondCheckpointId = 2; + OperatorSubtaskState snapshot = harness.snapshot(secondCheckpointId, ++timestamp); + assertFlinkManifests(2); + + // 3. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(firstCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertMaxCommittedCheckpointId(jobId, firstCheckpointId); + assertFlinkManifests(1); + + // 4. notifyCheckpointComplete for checkpoint#2 + harness.notifyOfCompletedCheckpoint(secondCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); + assertMaxCommittedCheckpointId(jobId, secondCheckpointId); + assertFlinkManifests(0); + } + } + + @TestTemplate + public void testDisorderedEventsBetweenCheckpoints() throws Exception { + // It's possible that two checkpoints happen in the following orders: + // 1. snapshotState for checkpoint#1; + // 2. snapshotState for checkpoint#2; + // 3. notifyCheckpointComplete for checkpoint#2; + // 4. notifyCheckpointComplete for checkpoint#1; + + long timestamp = 0; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + + harness.open(); + assertMaxCommittedCheckpointId(jobId, -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello"); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + + processElement(jobId, 1, harness, 1, OPERATOR_ID, dataFile1); + assertMaxCommittedCheckpointId(jobId, -1L); + + // 1. snapshotState for checkpoint#1 + long firstCheckpointId = 1; + harness.snapshot(firstCheckpointId, ++timestamp); + assertFlinkManifests(1); + + RowData row2 = SimpleDataUtil.createRowData(2, "world"); + DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); + processElement(jobId, 2, harness, 1, OPERATOR_ID, dataFile2); + assertMaxCommittedCheckpointId(jobId, -1L); + + // 2. snapshotState for checkpoint#2 + long secondCheckpointId = 2; + harness.snapshot(secondCheckpointId, ++timestamp); + assertFlinkManifests(2); + + // 3. notifyCheckpointComplete for checkpoint#2 + harness.notifyOfCompletedCheckpoint(secondCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); + assertMaxCommittedCheckpointId(jobId, secondCheckpointId); + assertFlinkManifests(0); + + // 4. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(firstCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); + assertMaxCommittedCheckpointId(jobId, secondCheckpointId); + assertFlinkManifests(0); + } + } + + @TestTemplate + public void testSingleCommit() throws Exception { + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness = getTestHarness()) { + testHarness.open(); + + long checkpointId = 1; + + RowData row1 = SimpleDataUtil.createRowData(1, "hello1"); + DataFile dataFile1 = writeDataFile("data-1-1", ImmutableList.of(row1)); + CommittableSummary committableSummary = + processElement(jobId, checkpointId, testHarness, 1, OPERATOR_ID, dataFile1); + + // Trigger commit + testHarness.notifyOfCompletedCheckpoint(checkpointId); + + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId, 1L); + + List output = transformsToStreamElement(testHarness.getOutput()); + + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary.getNumberOfCommittables()) + .hasPendingCommittables(0); + + SinkV2Assertions.assertThat(extractAndAssertCommittableWithLineage(output.get(1))) + .hasSubtaskId(0) + .hasCheckpointId(checkpointId); + } + + table.refresh(); + Snapshot currentSnapshot = table.snapshot(branch); + + assertThat(currentSnapshot.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, "1") + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1"); + } + + /** The data was not committed in the previous job. */ + @TestTemplate + public void testStateRestoreFromPreJobWithUncommitted() throws Exception { + String jobId1 = "jobId1"; + OperatorSubtaskState snapshot; + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + long timestamp = 0; + CommittableSummary committableSummary; + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + preJobTestHarness = getTestHarness()) { + + preJobTestHarness.open(); + + committableSummary = + processElement(jobId1, checkpointId, preJobTestHarness, 1, OPERATOR_ID, dataFileTest1); + + snapshot = preJobTestHarness.snapshot(checkpointId, ++timestamp); + + assertThat(preJobTestHarness.getOutput()).isEmpty(); + } + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId1, -1L); + + String jobId2 = "jobId2"; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + restored = getTestHarness()) { + restored.setup(committableMessageTypeSerializer); + restored.initializeState(snapshot); + restored.open(); + + // Previous committables are immediately committed if possible + List output = transformsToStreamElement(restored.getOutput()); + assertThat(output).hasSize(2); + + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary.getNumberOfCommittables()) + .hasPendingCommittables(0); + + SinkV2Assertions.assertThat(extractAndAssertCommittableWithLineage(output.get(1))) + .hasCheckpointId(0L) + .hasSubtaskId(0); + + table.refresh(); + + Snapshot currentSnapshot = table.snapshot(branch); + + assertThat(currentSnapshot.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1") + .containsEntry("flink.job-id", jobId1); + + checkpointId++; + CommittableSummary committableSummary2 = + processElement(jobId2, checkpointId, restored, 1, OPERATOR_ID, dataFileTest2); + + // Trigger commit + restored.notifyOfCompletedCheckpoint(checkpointId); + + List output2 = transformsToStreamElement(restored.getOutput()); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output2.get(0))) + .hasFailedCommittables(committableSummary2.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary2.getNumberOfCommittables()) + .hasPendingCommittables(0); + + SinkV2Assertions.assertThat(extractAndAssertCommittableWithLineage(output2.get(1))) + .hasCheckpointId(0L) + .hasSubtaskId(0); + } + + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId2, 1); + + table.refresh(); + Snapshot currentSnapshot2 = table.snapshot(branch); + + assertThat(currentSnapshot2.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount * 2)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "2") + .containsEntry("flink.job-id", jobId2); + } + + /** The data was committed in the previous job. */ + @TestTemplate + public void testStateRestoreFromPreJobWithCommitted() throws Exception { + String jobId1 = "jobId1"; + OperatorSubtaskState snapshot; + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + preJobTestHarness = getTestHarness()) { + + preJobTestHarness.open(); + + CommittableSummary committableSummary = + processElement(jobId1, checkpointId, preJobTestHarness, 1, OPERATOR_ID, dataFileTest1); + + assertFlinkManifests(1); + snapshot = preJobTestHarness.snapshot(checkpointId, 2L); + // commit snapshot + preJobTestHarness.notifyOfCompletedCheckpoint(checkpointId); + + List output = transformsToStreamElement(preJobTestHarness.getOutput()); + assertThat(output).hasSize(2); + + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary.getNumberOfCommittables()) + .hasPendingCommittables(0); + + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId1, checkpointId); + } + + table.refresh(); + long preJobSnapshotId = table.snapshot(branch).snapshotId(); + + String jobId2 = "jobId2"; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + restored = getTestHarness()) { + restored.setup(); + restored.initializeState(snapshot); + restored.open(); + + // Makes sure that data committed in the previous job is available in this job + List output2 = transformsToStreamElement(restored.getOutput()); + assertThat(output2).hasSize(2); + + table.refresh(); + long restoredSnapshotId = table.snapshot(branch).snapshotId(); + + assertThat(restoredSnapshotId) + .as("The table does not generate a new snapshot without data being committed.") + .isEqualTo(preJobSnapshotId); + + assertThat(table.snapshot(branch).summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1") + .containsEntry("flink.job-id", jobId1); + + // Commit new data file + checkpointId = 1; + CommittableSummary committableSummary2 = + processElement(jobId2, checkpointId, restored, 1, OPERATOR_ID, dataFileTest2); + + // Trigger commit + restored.notifyOfCompletedCheckpoint(checkpointId); + + List output3 = transformsToStreamElement(restored.getOutput()); + assertThat(output3).hasSize(4); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output3.get(0))) + .hasFailedCommittables(committableSummary2.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary2.getNumberOfCommittables()) + .hasPendingCommittables(0); + } + + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId2, 1L); + + table.refresh(); + Snapshot currentSnapshot2 = table.snapshot(branch); + assertThat(Long.parseLong(currentSnapshot2.summary().get(SnapshotSummary.TOTAL_RECORDS_PROP))) + .isEqualTo(dataFIleRowCount * 2); + + assertThat(currentSnapshot2.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount * 2)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "2") + .containsEntry("flink.job-id", jobId2); + } + + @TestTemplate + public void testStateRestoreFromCurrJob() throws Exception { + String jobId1 = "jobId1"; + CommittableSummary committableSummary; + OperatorSubtaskState snapshot; + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness = getTestHarness()) { + + testHarness.open(); + + committableSummary = + processElement(jobId1, checkpointId, testHarness, 1, OPERATOR_ID, dataFileTest1); + snapshot = testHarness.snapshot(checkpointId, 2L); + + assertThat(testHarness.getOutput()).isEmpty(); + } + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId1, -1L); + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + restored = getTestHarness()) { + + restored.setup(committableMessageTypeSerializer); + + restored.initializeState(snapshot); + restored.open(); + + // Previous committables are immediately committed if possible + List output = transformsToStreamElement(restored.getOutput()); + assertThat(output).hasSize(2); + + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary.getNumberOfCommittables()) + .hasPendingCommittables(0); + + table.refresh(); + Snapshot currentSnapshot = table.snapshot(branch); + + assertThat(currentSnapshot.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1") + .containsEntry("flink.job-id", jobId1); + + String jobId2 = "jobId2"; + checkpointId = 1; + CommittableSummary committableSummary2 = + processElement(jobId2, checkpointId, restored, 1, OPERATOR_ID, dataFileTest2); + + // Trigger commit + restored.notifyOfCompletedCheckpoint(checkpointId); + + List output2 = transformsToStreamElement(restored.getOutput()); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output2.get(0))) + .hasFailedCommittables(committableSummary2.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary2.getNumberOfCommittables()) + .hasPendingCommittables(0); + restored.close(); + + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId2, 1L); + + table.refresh(); + Snapshot currentSnapshot2 = table.snapshot(branch); + assertThat(currentSnapshot2.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount * 2)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "2") + .containsEntry("flink.job-id", jobId2); + } + } + + @TestTemplate + public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Exception { + // We've two steps in checkpoint: 1. snapshotState(ckp); 2. notifyCheckpointComplete(ckp). + // The Flink job should be able to restore from a checkpoint with only step#1 finished. + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + long timestamp = 0; + OperatorSubtaskState snapshot; + List expectedRows = Lists.newArrayList(); + + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId, -1L); + + RowData row = SimpleDataUtil.createRowData(1, "hello"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-1", ImmutableList.of(row)); + processElement(jobId, checkpointId, harness, 1, operatorId.toString(), dataFile); + + snapshot = harness.snapshot(++checkpointId, ++timestamp); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(), branch); + assertMaxCommittedCheckpointId(jobId, -1L); + assertFlinkManifests(1); + } + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + + harness.getStreamConfig().setOperatorID(operatorId); + harness.initializeState(snapshot); + harness.open(); + + // All flink manifests should be cleaned because it has committed the unfinished iceberg + // transaction. + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertMaxCommittedCheckpointId(jobId, operatorId.toString(), 0L); + + harness.snapshot(++checkpointId, ++timestamp); + // Did not write any new record, so it won't generate new manifest. + assertFlinkManifests(0); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(1); + + assertMaxCommittedCheckpointId(jobId, operatorId.toString(), 0); + + RowData row = SimpleDataUtil.createRowData(2, "world"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); + processElement(jobId, checkpointId, harness, 1, operatorId.toString(), dataFile); + + snapshot = harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(0); + } + + // Redeploying flink job from external checkpoint. + JobID newJobId = new JobID(); + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + harness.getStreamConfig().setOperatorID(operatorId); + harness.initializeState(snapshot); + harness.open(); + + // All flink manifests should be cleaned because it has committed the unfinished iceberg + // transaction. + assertFlinkManifests(0); + + assertMaxCommittedCheckpointId(newJobId.toString(), operatorId.toString(), -1); + assertMaxCommittedCheckpointId(jobId, operatorId.toString(), 2); + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(2); + + RowData row = SimpleDataUtil.createRowData(3, "foo"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-3", ImmutableList.of(row)); + processElement( + newJobId.toString(), checkpointId, harness, 1, operatorId.toString(), dataFile); + + harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(3); + assertMaxCommittedCheckpointId(newJobId.toString(), operatorId.toString(), 3); + } + } + + @TestTemplate + public void testStartAnotherJobToWriteSameTable() throws Exception { + long checkpointId = 1; + long timestamp = 0; + + List rows = Lists.newArrayList(); + List tableRows = Lists.newArrayList(); + + JobID oldJobId = new JobID(); + OperatorID oldOperatorId; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + + harness.open(); + oldOperatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(oldJobId.toString(), oldOperatorId.toString(), -1L); + + for (int i = 1; i <= 3; i++) { + rows.add(SimpleDataUtil.createRowData(i, "hello" + i)); + tableRows.addAll(rows); + + DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); + processElement( + oldJobId.toString(), ++checkpointId, harness, 1, oldOperatorId.toString(), dataFile); + harness.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(i); + assertMaxCommittedCheckpointId(oldJobId.toString(), oldOperatorId.toString(), checkpointId); + } + } + + // The new started job will start with checkpoint = 1 again. + checkpointId = 1; + JobID newJobId = new JobID(); + OperatorID newOperatorId; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + harness.open(); + newOperatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(3); + assertMaxCommittedCheckpointId(oldJobId.toString(), oldOperatorId.toString(), 4); + assertMaxCommittedCheckpointId(newJobId.toString(), newOperatorId.toString(), -1); + + rows.add(SimpleDataUtil.createRowData(2, "world")); + tableRows.addAll(rows); + + DataFile dataFile = writeDataFile("data-new-1", rows); + processElement( + newJobId.toString(), checkpointId, harness, 1, newOperatorId.toString(), dataFile); + harness.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(4); + assertMaxCommittedCheckpointId(newJobId.toString(), newOperatorId.toString(), checkpointId); + } + } + + @TestTemplate + public void testMultipleJobsWriteSameTable() throws Exception { + long timestamp = 0; + List tableRows = Lists.newArrayList(); + + JobID[] jobs = new JobID[] {new JobID(), new JobID(), new JobID()}; + OperatorID[] operatorIds = + new OperatorID[] {new OperatorID(), new OperatorID(), new OperatorID()}; + for (int i = 0; i < 20; i++) { + int jobIndex = i % 3; + int checkpointId = i / 3; + JobID jobID = jobs[jobIndex]; + OperatorID operatorId = operatorIds[jobIndex]; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + harness.getStreamConfig().setOperatorID(operatorId); + + harness.open(); + + assertSnapshotSize(i); + assertMaxCommittedCheckpointId( + jobID.toString(), operatorId.toString(), checkpointId == 0 ? -1 : checkpointId - 1); + + List rows = Lists.newArrayList(SimpleDataUtil.createRowData(i, "word-" + i)); + tableRows.addAll(rows); + + DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); + + processElement(jobID.toString(), checkpointId, harness, 1, operatorId.toString(), dataFile); + + harness.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(i + 1); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId.toString(), checkpointId); + } + } + } + + @TestTemplate + public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + long timestamp = 0; + List expectedRows = Lists.newArrayList(); + OperatorSubtaskState snapshot1; + OperatorSubtaskState snapshot2; + + JobID jobID = new JobID(); + OperatorID operatorId1 = new OperatorID(); + OperatorID operatorId2 = new OperatorID(); + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness1 = getTestHarness()) { + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness2 = getTestHarness()) { + harness1.getStreamConfig().setOperatorID(operatorId1); + harness1.setup(); + harness1.open(); + harness2.getStreamConfig().setOperatorID(operatorId2); + harness2.setup(); + harness2.open(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId1.toString(), -1L); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId2.toString(), -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello1"); + expectedRows.add(row1); + DataFile dataFile1 = writeDataFile("data-1-1", ImmutableList.of(row1)); + processElement( + jobID.toString(), checkpointId, harness1, 1, operatorId1.toString(), dataFile1); + + snapshot1 = harness1.snapshot(checkpointId, ++timestamp); + + RowData row2 = SimpleDataUtil.createRowData(1, "hello2"); + expectedRows.add(row2); + DataFile dataFile2 = writeDataFile("data-1-2", ImmutableList.of(row2)); + processElement( + jobID.toString(), checkpointId, harness2, 1, operatorId2.toString(), dataFile2); + + snapshot2 = harness2.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(2); + + // Only notify one of the committers + harness1.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(1); + + // Only the first row is committed at this point + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId1.toString(), checkpointId); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId2.toString(), -1); + } + } + + // Restore from the given snapshot + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness1 = getTestHarness(); + OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness2 = getTestHarness()) { + harness1.getStreamConfig().setOperatorID(operatorId1); + harness1.setup(); + harness1.initializeState(snapshot1); + harness1.open(); + + harness2.getStreamConfig().setOperatorID(operatorId2); + harness2.setup(); + harness2.initializeState(snapshot2); + harness2.open(); + + // All flink manifests should be cleaned because it has committed the unfinished iceberg + // transaction. + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId1.toString(), checkpointId); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId2.toString(), checkpointId); + + RowData row1 = SimpleDataUtil.createRowData(2, "world1"); + expectedRows.add(row1); + DataFile dataFile1 = writeDataFile("data-2-1", ImmutableList.of(row1)); + + checkpointId++; + processElement( + jobID.toString(), checkpointId, harness1, 1, operatorId1.toString(), dataFile1); + + harness1.snapshot(checkpointId, ++timestamp); + + RowData row2 = SimpleDataUtil.createRowData(2, "world2"); + expectedRows.add(row2); + DataFile dataFile2 = writeDataFile("data-2-2", ImmutableList.of(row2)); + processElement( + jobID.toString(), checkpointId, harness2, 1, operatorId2.toString(), dataFile2); + + harness2.snapshot(checkpointId, ++timestamp); + + assertFlinkManifests(2); + + harness1.notifyOfCompletedCheckpoint(checkpointId); + harness2.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(4); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId1.toString(), checkpointId); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId2.toString(), checkpointId); + } + } + + @TestTemplate + public void testFlinkManifests() throws Exception { + long timestamp = 0; + long checkpoint = 1; + + JobID jobID = new JobID(); + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertMaxCommittedCheckpointId(jobID.toString(), operatorId.toString(), -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello"); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + + // harness.processElement(of(dataFile1), ++timestamp); + processElement(jobID.toString(), checkpoint, harness, 1, operatorId.toString(), dataFile1); + + assertMaxCommittedCheckpointId(jobID.toString(), operatorId.toString(), -1L); + + // 1. snapshotState for checkpoint#1 + harness.snapshot(checkpoint, ++timestamp); + List manifestPaths = assertFlinkManifests(1); + Path manifestPath = manifestPaths.get(0); + assertThat(manifestPath.getFileName()) + .asString() + .isEqualTo( + String.format("%s-%s-%05d-%d-%d-%05d.avro", jobID, operatorId, 0, 0, checkpoint, 1)); + // + // 2. Read the data files from manifests and assert. + List dataFiles = + FlinkManifestUtil.readDataFiles( + createTestingManifestFile(manifestPath), table.io(), table.specs()); + assertThat(dataFiles).hasSize(1); + TestHelpers.assertEquals(dataFile1, dataFiles.get(0)); + + // 3. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(checkpoint); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId.toString(), checkpoint); + assertFlinkManifests(0); + } + } + + @TestTemplate + public void testHandleEndInput() throws Exception { + assumeThat(isStreamingMode).as("Only support batch mode").isFalse(); + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness = getTestHarness()) { + + testHarness.open(); + + long checkpointId = Long.MAX_VALUE; + processElement(jobId, checkpointId, testHarness, 1, OPERATOR_ID, dataFileTest1); + + testHarness.endInput(); + + assertMaxCommittedCheckpointId(jobId, OPERATOR_ID, Long.MAX_VALUE); + + List output = transformsToStreamElement(testHarness.getOutput()); + assertThat(output).hasSize(2); + + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasCheckpointId(checkpointId) + .hasPendingCommittables(0) + .hasOverallCommittables(1) + .hasFailedCommittables(0); + + // endInput is idempotent + testHarness.endInput(); + assertThat(testHarness.getOutput()).hasSize(2); + } + } + + @TestTemplate + public void testDeleteFiles() throws Exception { + + assumeThat(formatVersion).as("Only support delete in format v2").isGreaterThanOrEqualTo(2); + + FileAppenderFactory appenderFactory = createDeletableAppenderFactory(); + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness = getTestHarness()) { + + testHarness.open(); + + long checkpointId = 1; + RowData row1 = SimpleDataUtil.createInsert(1, "aaa"); + DataFile dataFile1 = writeDataFile("data-file-1", ImmutableList.of(row1)); + processElement(jobId, checkpointId, testHarness, 1, OPERATOR_ID, dataFile1); + + // testHarness.snapshot(checkpointId, 0); + testHarness.notifyOfCompletedCheckpoint(checkpointId); + + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId, checkpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + + List output = transformsToStreamElement(testHarness.getOutput()); + assertThat(output).hasSize(2); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasCheckpointId(checkpointId) + .hasPendingCommittables(0) + .hasOverallCommittables(1) + .hasFailedCommittables(0); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + + // The 2. commit + checkpointId = 2; + RowData row2 = SimpleDataUtil.createInsert(2, "bbb"); + DataFile dataFile2 = writeDataFile("data-file-2", ImmutableList.of(row2)); + + RowData row3 = SimpleDataUtil.createInsert(3, "ccc"); + DataFile dataFile3 = writeDataFile("data-file-3", ImmutableList.of(row3)); + processElement(jobId, checkpointId, testHarness, 2, OPERATOR_ID, dataFile2, dataFile3); + + // testHarness.snapshot(checkpointId, 1); + testHarness.notifyOfCompletedCheckpoint(checkpointId); + + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId, checkpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2, row3), branch); + + List output2 = transformsToStreamElement(testHarness.getOutput()); + assertThat(output2).hasSize(2 + 2); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output2.get(2))) + .hasCheckpointId(checkpointId) + .hasPendingCommittables(0) + .hasOverallCommittables(1) + .hasFailedCommittables(0); + + // The 3. commit + checkpointId = 3; + RowData delete1 = SimpleDataUtil.createDelete(1, "aaa"); + DeleteFile deleteFile1 = + writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete1)); + RowData row4 = SimpleDataUtil.createInsert(4, "ddd"); + DataFile dataFile4 = writeDataFile("data-file-4", ImmutableList.of(row4)); + + RowData row5 = SimpleDataUtil.createInsert(5, "eee"); + DataFile dataFile5 = writeDataFile("data-file-5", ImmutableList.of(row5)); + WriteResult withRecord4 = + WriteResult.builder() + .addDataFiles(dataFile4, dataFile5) + .addDeleteFiles(deleteFile1) + .build(); + processElement(withRecord4, jobId, checkpointId, testHarness, 2, OPERATOR_ID); + + // testHarness.snapshot(checkpointId, 3); + testHarness.notifyOfCompletedCheckpoint(checkpointId); + + assertSnapshotSize(3); + assertMaxCommittedCheckpointId(jobId, checkpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row2, row3, row4, row5), branch); + + List output3 = transformsToStreamElement(testHarness.getOutput()); + assertThat(output3).hasSize(2 + 2 + 2); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output3.get(4))) + .hasCheckpointId(checkpointId) + .hasPendingCommittables(0) + .hasOverallCommittables(1) + .hasFailedCommittables(0); + } + } + + private ManifestFile createTestingManifestFile(Path manifestPath) { + return new GenericManifestFile( + manifestPath.toAbsolutePath().toString(), + manifestPath.toFile().length(), + 0, + ManifestContent.DATA, + 0, + 0, + 0L, + 0, + 0, + 0, + 0, + 0, + 0, + null, + null); + } + + private IcebergWriteAggregator buildIcebergWriteAggregator(String myJobId, String operatorId) { + IcebergWriteAggregator icebergWriteAggregator = spy(new IcebergWriteAggregator(tableLoader)); + StreamTask ctx = mock(StreamTask.class); + Environment env = mock(Environment.class); + StreamingRuntimeContext streamingRuntimeContext = mock(StreamingRuntimeContext.class); + TaskInfo taskInfo = mock(TaskInfo.class); + JobID myJobID = mock(JobID.class); + OperatorID operatorID = mock(OperatorID.class); + doReturn(myJobId).when(myJobID).toString(); + doReturn(myJobID).when(env).getJobID(); + doReturn(env).when(ctx).getEnvironment(); + doReturn(ctx).when(icebergWriteAggregator).getContainingTask(); + doReturn(operatorId).when(operatorID).toString(); + doReturn(operatorID).when(icebergWriteAggregator).getOperatorID(); + doReturn(0).when(taskInfo).getAttemptNumber(); + doReturn(taskInfo).when(streamingRuntimeContext).getTaskInfo(); + doReturn(streamingRuntimeContext).when(icebergWriteAggregator).getRuntimeContext(); + + try { + icebergWriteAggregator.open(); + } catch (Exception e) { + throw new RuntimeException(e); + } + return icebergWriteAggregator; + } + + private CommittableSummary processElement( + WriteResult withRecord, + String myJobId, + long checkpointId, + OneInputStreamOperatorTestHarness testHarness, + int subTaskId, + String operatorId) + throws Exception { + + IcebergCommittable commit = + new IcebergCommittable( + buildIcebergWriteAggregator(myJobId, operatorId) + .writeToManifest(Lists.newArrayList(withRecord), checkpointId), + myJobId, + operatorId, + checkpointId); + + CommittableSummary committableSummary = + new CommittableSummary<>(subTaskId, 1, checkpointId, 1, 1, 0); + testHarness.processElement(new StreamRecord<>(committableSummary)); + + CommittableWithLineage committable = + new CommittableWithLineage<>(commit, checkpointId, subTaskId); + testHarness.processElement(new StreamRecord<>(committable)); + + return committableSummary; + } + + private CommittableSummary processElement( + String myJobID, + long checkpointId, + OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness, + int subTaskId, + String operatorId, + DataFile... dataFile) + throws Exception { + WriteResult withRecord = WriteResult.builder().addDataFiles(dataFile).build(); + return processElement(withRecord, myJobID, checkpointId, testHarness, subTaskId, operatorId); + } + + private FileAppenderFactory createDeletableAppenderFactory() { + int[] equalityFieldIds = + new int[] { + table.schema().findField("id").fieldId(), table.schema().findField("data").fieldId() + }; + return new FlinkAppenderFactory( + table, + table.schema(), + FlinkSchemaUtil.convert(table.schema()), + table.properties(), + table.spec(), + equalityFieldIds, + table.schema(), + null); + } + + private List assertFlinkManifests(int expectedCount) throws IOException { + List manifests = + Files.list(flinkManifestFolder.toPath()) + .filter(p -> !p.toString().endsWith(".crc")) + .collect(Collectors.toList()); + assertThat(manifests).hasSize(expectedCount); + return manifests; + } + + private DataFile writeDataFile(String filename, List rows) throws IOException { + return SimpleDataUtil.writeFile( + table, + table.schema(), + table.spec(), + new Configuration(), + table.location(), + FileFormat.PARQUET.addExtension(filename), + rows); + } + + private DeleteFile writeEqDeleteFile( + FileAppenderFactory appenderFactory, String filename, List deletes) + throws IOException { + return SimpleDataUtil.writeEqDeleteFile( + table, FileFormat.PARQUET, filename, appenderFactory, deletes); + } + + private OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + getTestHarness() throws Exception { + IcebergSink sink = + IcebergSink.forRowData(null).table(table).toBranch(branch).tableLoader(tableLoader).build(); + + OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness = + new OneInputStreamOperatorTestHarness<>( + new CommitterOperatorFactory<>(sink, !isStreamingMode, true)); + testHarness.setup(committableMessageTypeSerializer); + return testHarness; + } + + // ------------------------------- Utility Methods -------------------------------- + + private IcebergCommitter getCommitter() { + IcebergFilesCommitterMetrics metric = mock(IcebergFilesCommitterMetrics.class); + return new IcebergCommitter( + tableLoader, + branch, + Collections.singletonMap("flink.test", TestIcebergCommitter.class.getName()), + false, + 10, + "sinkId", + metric); + } + + private Committer.CommitRequest buildCommitRequestFor( + String myJobID, long checkpoint, Collection writeResults) throws IOException { + IcebergCommittable commit = + new IcebergCommittable( + buildIcebergWriteAggregator(myJobID, OPERATOR_ID) + .writeToManifest(writeResults, checkpoint), + myJobID, + OPERATOR_ID, + checkpoint); + + CommittableWithLineage committableWithLineage = + new CommittableWithLineage(commit, checkpoint, 1); + Committer.CommitRequest commitRequest = mock(Committer.CommitRequest.class); + + doReturn(committableWithLineage.getCommittable()).when(commitRequest).getCommittable(); + + return commitRequest; + } + + private WriteResult of(DataFile dataFile) { + return WriteResult.builder().addDataFiles(dataFile).build(); + } + + private void assertMaxCommittedCheckpointId(String myJobID, String operatorId, long expectedId) { + table.refresh(); + long actualId = SinkUtil.getMaxCommittedCheckpointId(table, myJobID, operatorId, branch); + assertThat(actualId).isEqualTo(expectedId); + } + + private void assertMaxCommittedCheckpointId(String myJobID, long expectedId) { + assertMaxCommittedCheckpointId(myJobID, OPERATOR_ID, expectedId); + } + + private void assertSnapshotSize(int expectedSnapshotSize) { + table.refresh(); + assertThat(table.snapshots()).hasSize(expectedSnapshotSize); + } + + private static ByteBuffer longToBuffer(long value) { + return ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN).putLong(0, value); + } + + private static class TestCommittableMessageTypeSerializer + extends TypeSerializer> { + + CommittableMessageSerializer serializer = + new CommittableMessageSerializer<>(new IcebergCommittableSerializer()); + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer> duplicate() { + return null; + } + + @Override + public CommittableMessage createInstance() { + return null; + } + + @Override + public CommittableMessage copy( + CommittableMessage from) { + return from; + } + + @Override + public CommittableMessage copy( + CommittableMessage from, CommittableMessage reuse) { + return from; + } + + @Override + public int getLength() { + return 0; + } + + @Override + public void serialize(CommittableMessage record, DataOutputView target) + throws IOException { + byte[] serialize = serializer.serialize(record); + target.writeInt(serialize.length); + target.write(serialize); + } + + @Override + public CommittableMessage deserialize(DataInputView source) + throws IOException { + int length = source.readInt(); + byte[] bytes = new byte[length]; + source.read(bytes); + return serializer.deserialize(1, bytes); + } + + @Override + public CommittableMessage deserialize( + CommittableMessage reuse, DataInputView source) throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + CommittableMessage deserialize = deserialize(source); + serialize(deserialize, target); + } + + @Override + public boolean equals(Object obj) { + return false; + } + + @Override + public int hashCode() { + return 0; + } + + @Override + public TypeSerializerSnapshot> snapshotConfiguration() { + return null; + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java index 948c7b31430c..7808771d9887 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java @@ -129,7 +129,8 @@ public void testCommitTxnWithoutDataFiles() throws Exception { long timestamp = 0; JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -139,8 +140,7 @@ public void testCommitTxnWithoutDataFiles() throws Exception { assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // It's better to advance the max-committed-checkpoint-id in iceberg snapshot, so that the - // future flink job - // failover won't fail. + // future flink job failover won't fail. for (int i = 1; i <= 3; i++) { harness.snapshot(++checkpointId, ++timestamp); assertFlinkManifests(0); @@ -161,7 +161,8 @@ public void testMaxContinuousEmptyCommits() throws Exception { JobID jobId = new JobID(); long checkpointId = 0; long timestamp = 0; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); @@ -176,8 +177,8 @@ public void testMaxContinuousEmptyCommits() throws Exception { } } - private WriteResult of(DataFile dataFile) { - return WriteResult.builder().addDataFiles(dataFile).build(); + private FlinkWriteResult of(long checkpointId, DataFile dataFile) { + return new FlinkWriteResult(checkpointId, WriteResult.builder().addDataFiles(dataFile).build()); } @TestTemplate @@ -193,7 +194,8 @@ public void testCommitTxn() throws Exception { JobID jobID = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobID)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobID)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -204,7 +206,7 @@ public void testCommitTxn() throws Exception { for (int i = 1; i <= 3; i++) { RowData rowData = SimpleDataUtil.createRowData(i, "hello" + i); DataFile dataFile = writeDataFile("data-" + i, ImmutableList.of(rowData)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(i, dataFile), ++timestamp); rows.add(rowData); harness.snapshot(i, ++timestamp); @@ -233,7 +235,8 @@ public void testOrderedEventsBetweenCheckpoints() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -243,21 +246,21 @@ public void testOrderedEventsBetweenCheckpoints() throws Exception { RowData row1 = SimpleDataUtil.createRowData(1, "hello"); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + long firstCheckpointId = 1; + harness.processElement(of(firstCheckpointId, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 - long firstCheckpointId = 1; harness.snapshot(firstCheckpointId, ++timestamp); assertFlinkManifests(1); RowData row2 = SimpleDataUtil.createRowData(2, "world"); DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); - harness.processElement(of(dataFile2), ++timestamp); + long secondCheckpointId = 2; + harness.processElement(of(secondCheckpointId, dataFile2), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 2. snapshotState for checkpoint#2 - long secondCheckpointId = 2; harness.snapshot(secondCheckpointId, ++timestamp); assertFlinkManifests(2); @@ -286,7 +289,8 @@ public void testDisorderedEventsBetweenCheckpoints() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -296,21 +300,21 @@ public void testDisorderedEventsBetweenCheckpoints() throws Exception { RowData row1 = SimpleDataUtil.createRowData(1, "hello"); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + long firstCheckpointId = 1; + harness.processElement(of(firstCheckpointId, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 - long firstCheckpointId = 1; harness.snapshot(firstCheckpointId, ++timestamp); assertFlinkManifests(1); RowData row2 = SimpleDataUtil.createRowData(2, "world"); DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); - harness.processElement(of(dataFile2), ++timestamp); + long secondCheckpointId = 2; + harness.processElement(of(secondCheckpointId, dataFile2), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 2. snapshotState for checkpoint#2 - long secondCheckpointId = 2; harness.snapshot(secondCheckpointId, ++timestamp); assertFlinkManifests(2); @@ -337,7 +341,8 @@ public void testRecoveryFromValidSnapshot() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -349,8 +354,8 @@ public void testRecoveryFromValidSnapshot() throws Exception { expectedRows.add(row); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row)); - harness.processElement(of(dataFile1), ++timestamp); - snapshot = harness.snapshot(++checkpointId, ++timestamp); + harness.processElement(of(++checkpointId, dataFile1), ++timestamp); + snapshot = harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -362,7 +367,8 @@ public void testRecoveryFromValidSnapshot() throws Exception { } // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.initializeState(snapshot); @@ -375,9 +381,9 @@ public void testRecoveryFromValidSnapshot() throws Exception { RowData row = SimpleDataUtil.createRowData(2, "world"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -400,7 +406,8 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except List expectedRows = Lists.newArrayList(); JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -411,15 +418,16 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(1, "hello"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-1", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - snapshot = harness.snapshot(++checkpointId, ++timestamp); + snapshot = harness.snapshot(checkpointId, ++timestamp); SimpleDataUtil.assertTableRows(table, ImmutableList.of(), branch); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); assertFlinkManifests(1); } - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.initializeState(snapshot); @@ -446,15 +454,15 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(2, "world"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - snapshot = harness.snapshot(++checkpointId, ++timestamp); + snapshot = harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); } // Redeploying flink job from external checkpoint. JobID newJobId = new JobID(); - try (OneInputStreamOperatorTestHarness harness = + try (OneInputStreamOperatorTestHarness harness = createStreamSink(newJobId)) { harness.setup(); harness.initializeState(snapshot); @@ -473,9 +481,9 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(3, "foo"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-3", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -496,7 +504,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { JobID oldJobId = new JobID(); OperatorID oldOperatorId; - try (OneInputStreamOperatorTestHarness harness = + try (OneInputStreamOperatorTestHarness harness = createStreamSink(oldJobId)) { harness.setup(); harness.open(); @@ -510,8 +518,8 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { tableRows.addAll(rows); DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); - harness.processElement(of(dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -528,7 +536,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { timestamp = 0; JobID newJobId = new JobID(); OperatorID newOperatorId; - try (OneInputStreamOperatorTestHarness harness = + try (OneInputStreamOperatorTestHarness harness = createStreamSink(newJobId)) { harness.setup(); harness.open(); @@ -542,8 +550,8 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { tableRows.addAll(rows); DataFile dataFile = writeDataFile("data-new-1", rows); - harness.processElement(of(dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -567,7 +575,8 @@ public void testMultipleJobsWriteSameTable() throws Exception { int checkpointId = i / 3; JobID jobId = jobs[jobIndex]; OperatorID operatorId = operatorIds[jobIndex]; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.open(); @@ -579,7 +588,7 @@ public void testMultipleJobsWriteSameTable() throws Exception { tableRows.addAll(rows); DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId + 1, dataFile), ++timestamp); harness.snapshot(checkpointId + 1, ++timestamp); assertFlinkManifests(1); @@ -603,8 +612,10 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { JobID jobId = new JobID(); OperatorID operatorId1 = new OperatorID(); OperatorID operatorId2 = new OperatorID(); - try (OneInputStreamOperatorTestHarness harness1 = createStreamSink(jobId); - OneInputStreamOperatorTestHarness harness2 = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness1 = + createStreamSink(jobId); + OneInputStreamOperatorTestHarness harness2 = + createStreamSink(jobId)) { harness1.getStreamConfig().setOperatorID(operatorId1); harness1.setup(); harness1.open(); @@ -620,14 +631,14 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { expectedRows.add(row1); DataFile dataFile1 = writeDataFile("data-1-1", ImmutableList.of(row1)); - harness1.processElement(of(dataFile1), ++timestamp); - snapshot1 = harness1.snapshot(++checkpointId, ++timestamp); + harness1.processElement(of(++checkpointId, dataFile1), ++timestamp); + snapshot1 = harness1.snapshot(checkpointId, ++timestamp); RowData row2 = SimpleDataUtil.createRowData(1, "hello2"); expectedRows.add(row2); DataFile dataFile2 = writeDataFile("data-1-2", ImmutableList.of(row2)); - harness2.processElement(of(dataFile2), ++timestamp); + harness2.processElement(of(checkpointId, dataFile2), ++timestamp); snapshot2 = harness2.snapshot(checkpointId, ++timestamp); assertFlinkManifests(2); @@ -643,8 +654,10 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { } // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness harness1 = createStreamSink(jobId); - OneInputStreamOperatorTestHarness harness2 = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness1 = + createStreamSink(jobId); + OneInputStreamOperatorTestHarness harness2 = + createStreamSink(jobId)) { harness1.getStreamConfig().setOperatorID(operatorId1); harness1.setup(); harness1.initializeState(snapshot1); @@ -668,13 +681,13 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { expectedRows.add(row1); DataFile dataFile1 = writeDataFile("data-2-1", ImmutableList.of(row1)); - harness1.processElement(of(dataFile1), ++timestamp); - harness1.snapshot(++checkpointId, ++timestamp); + harness1.processElement(of(++checkpointId, dataFile1), ++timestamp); + harness1.snapshot(checkpointId, ++timestamp); RowData row2 = SimpleDataUtil.createRowData(2, "world2"); expectedRows.add(row2); DataFile dataFile2 = writeDataFile("data-2-2", ImmutableList.of(row2)); - harness2.processElement(of(dataFile2), ++timestamp); + harness2.processElement(of(checkpointId, dataFile2), ++timestamp); harness2.snapshot(checkpointId, ++timestamp); assertFlinkManifests(2); @@ -694,7 +707,8 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { public void testBoundedStream() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -706,13 +720,14 @@ public void testBoundedStream() throws Exception { List tableRows = Lists.newArrayList(SimpleDataUtil.createRowData(1, "word-1")); DataFile dataFile = writeDataFile("data-1", tableRows); - harness.processElement(of(dataFile), 1); + harness.processElement(of(IcebergStreamWriter.END_INPUT_CHECKPOINT_ID, dataFile), 1); ((BoundedOneInput) harness.getOneInputOperator()).endInput(); assertFlinkManifests(0); SimpleDataUtil.assertTableRows(table, tableRows, branch); assertSnapshotSize(1); - assertMaxCommittedCheckpointId(jobId, operatorId, Long.MAX_VALUE); + assertMaxCommittedCheckpointId( + jobId, operatorId, IcebergStreamWriter.END_INPUT_CHECKPOINT_ID); assertThat(SimpleDataUtil.latestSnapshot(table, branch).summary()) .containsEntry("flink.test", TestIcebergFilesCommitter.class.getName()); } @@ -725,7 +740,8 @@ public void testFlinkManifests() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -735,7 +751,7 @@ public void testFlinkManifests() throws Exception { RowData row1 = SimpleDataUtil.createRowData(1, "hello"); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + harness.processElement(of(checkpoint, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 @@ -775,7 +791,8 @@ public void testDeleteFiles() throws Exception { OperatorID operatorId; FileAppenderFactory appenderFactory = createDeletableAppenderFactory(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -784,7 +801,7 @@ public void testDeleteFiles() throws Exception { RowData row1 = SimpleDataUtil.createInsert(1, "aaa"); DataFile dataFile1 = writeDataFile("data-file-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + harness.processElement(of(checkpoint, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 @@ -816,13 +833,15 @@ public void testDeleteFiles() throws Exception { RowData delete1 = SimpleDataUtil.createDelete(1, "aaa"); DeleteFile deleteFile1 = writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete1)); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); harness.processElement( - WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile1).build(), + new FlinkWriteResult( + ++checkpoint, + WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile1).build()), ++timestamp); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); // 5. snapshotState for checkpoint#2 - harness.snapshot(++checkpoint, ++timestamp); + harness.snapshot(checkpoint, ++timestamp); assertFlinkManifests(2); // 6. notifyCheckpointComplete for checkpoint#2 @@ -846,7 +865,8 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { OperatorID operatorId; FileAppenderFactory appenderFactory = createDeletableAppenderFactory(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -860,7 +880,9 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { DeleteFile deleteFile1 = writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete3)); harness.processElement( - WriteResult.builder().addDataFiles(dataFile1).addDeleteFiles(deleteFile1).build(), + new FlinkWriteResult( + checkpoint, + WriteResult.builder().addDataFiles(dataFile1).addDeleteFiles(deleteFile1).build()), ++timestamp); // The 1th snapshotState. @@ -872,11 +894,13 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { DeleteFile deleteFile2 = writeEqDeleteFile(appenderFactory, "delete-file-2", ImmutableList.of(delete2)); harness.processElement( - WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile2).build(), + new FlinkWriteResult( + ++checkpoint, + WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile2).build()), ++timestamp); // The 2nd snapshotState. - harness.snapshot(++checkpoint, ++timestamp); + harness.snapshot(checkpoint, ++timestamp); // Notify the 2nd snapshot to complete. harness.notifyOfCompletedCheckpoint(checkpoint); @@ -887,6 +911,79 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { } } + /** + * The testcase is to simulate upserting to an Iceberg V2 table, and facing the following + * scenario: + * + *
    + *
  • A specific row is updated + *
  • The prepareSnapshotPreBarrier triggered + *
  • Checkpoint failed for reasons outside of the Iceberg connector + *
  • The specific row is updated again in the second checkpoint as well + *
  • Second snapshot is triggered, and finished + *
+ * + *

Previously the files from the 2 snapshots were committed in a single Iceberg commit, as a + * results duplicate rows were created in the table. + * + * @throws Exception Exception + */ + @TestTemplate + public void testCommitMultipleCheckpointsForV2Table() throws Exception { + assumeThat(formatVersion) + .as("Only support equality-delete in format v2 or later.") + .isGreaterThan(1); + + long timestamp = 0; + long checkpoint = 10; + + JobID jobId = new JobID(); + OperatorID operatorId; + + FileAppenderFactory appenderFactory = + new FlinkAppenderFactory( + table, + table.schema(), + FlinkSchemaUtil.convert(table.schema()), + table.properties(), + table.spec(), + new int[] {table.schema().findField("id").fieldId()}, + table.schema(), + null); + + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + RowData insert1 = null; + RowData insert2 = null; + for (int i = 1; i <= 3; i++) { + insert1 = SimpleDataUtil.createInsert(1, "aaa" + i); + insert2 = SimpleDataUtil.createInsert(2, "bbb" + i); + DataFile dataFile = writeDataFile("data-file-" + i, ImmutableList.of(insert1, insert2)); + DeleteFile deleteFile = + writeEqDeleteFile( + appenderFactory, "delete-file-" + i, ImmutableList.of(insert1, insert2)); + harness.processElement( + new FlinkWriteResult( + ++checkpoint, + WriteResult.builder().addDataFiles(dataFile).addDeleteFiles(deleteFile).build()), + ++timestamp); + } + + harness.snapshot(checkpoint, ++timestamp); + harness.notifyOfCompletedCheckpoint(checkpoint); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(insert1, insert2), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); + assertFlinkManifests(0); + assertThat(table.snapshots()).hasSize(3); + } + } + @TestTemplate public void testSpecEvolution() throws Exception { long timestamp = 0; @@ -899,7 +996,8 @@ public void testSpecEvolution() throws Exception { DataFile dataFile; int specId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -910,7 +1008,7 @@ public void testSpecEvolution() throws Exception { RowData rowData = SimpleDataUtil.createRowData(checkpointId, "hello" + checkpointId); // table unpartitioned dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(rowData)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId, dataFile), ++timestamp); rows.add(rowData); harness.snapshot(checkpointId, ++timestamp); @@ -929,7 +1027,7 @@ public void testSpecEvolution() throws Exception { rowData = SimpleDataUtil.createRowData(checkpointId, "hello" + checkpointId); // write data with old partition spec dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(rowData), oldSpec, null); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId, dataFile), ++timestamp); rows.add(rowData); snapshot = harness.snapshot(checkpointId, ++timestamp); @@ -947,7 +1045,8 @@ public void testSpecEvolution() throws Exception { } // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.initializeState(snapshot); @@ -963,7 +1062,7 @@ public void testSpecEvolution() throws Exception { partition.set(0, checkpointId); dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(row), table.spec(), partition); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId, dataFile), ++timestamp); rows.add(row); harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); @@ -1079,8 +1178,8 @@ private DataFile writeDataFile( private void assertMaxCommittedCheckpointId(JobID jobID, OperatorID operatorID, long expectedId) { table.refresh(); long actualId = - IcebergFilesCommitter.getMaxCommittedCheckpointId( - table, jobID.toString(), operatorID.toHexString(), branch); + SinkUtil.getMaxCommittedCheckpointId( + table, jobID.toString(), operatorID.toString(), branch); assertThat(actualId).isEqualTo(expectedId); } @@ -1089,7 +1188,7 @@ private void assertSnapshotSize(int expectedSnapshotSize) { assertThat(table.snapshots()).hasSize(expectedSnapshotSize); } - private OneInputStreamOperatorTestHarness createStreamSink(JobID jobID) + private OneInputStreamOperatorTestHarness createStreamSink(JobID jobID) throws Exception { TestOperatorFactory factory = TestOperatorFactory.of(table.location(), branch, table.spec()); return new OneInputStreamOperatorTestHarness<>(factory, createEnvironment(jobID)); @@ -1109,7 +1208,7 @@ private static MockEnvironment createEnvironment(JobID jobID) { } private static class TestOperatorFactory extends AbstractStreamOperatorFactory - implements OneInputStreamOperatorFactory { + implements OneInputStreamOperatorFactory { private final String tablePath; private final String branch; private final PartitionSpec spec; diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSink.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSink.java new file mode 100644 index 000000000000..7f355c1e8403 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSink.java @@ -0,0 +1,436 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.sink.IcebergSink.Builder; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergSink extends TestFlinkIcebergSinkBase { + + private TableLoader tableLoader; + + @Parameter(index = 0) + private FileFormat format; + + @Parameter(index = 1) + private int parallelism; + + @Parameter(index = 2) + private boolean partitioned; + + @Parameters(name = "format={0}, parallelism={1}, partitioned={2}") + public static Object[][] parameters() { + return new Object[][] { + {FileFormat.AVRO, 1, true}, + {FileFormat.AVRO, 1, false}, + {FileFormat.AVRO, 2, true}, + {FileFormat.AVRO, 2, false}, + {FileFormat.ORC, 1, true}, + {FileFormat.ORC, 1, false}, + {FileFormat.ORC, 2, true}, + {FileFormat.ORC, 2, false}, + {FileFormat.PARQUET, 1, true}, + {FileFormat.PARQUET, 1, false}, + {FileFormat.PARQUET, 2, true}, + {FileFormat.PARQUET, 2, false} + }; + } + + @BeforeEach + void before() throws IOException { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + void testWriteRowData() throws Exception { + List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) + .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); + + IcebergSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream"); + + // Assert the iceberg table's records. + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } + + @TestTemplate + void testWriteRow() throws Exception { + testWriteRow(null, DistributionMode.NONE); + } + + @TestTemplate + void testWriteRowWithTableSchema() throws Exception { + testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); + } + + @TestTemplate + void testJobNoneDistributeMode() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + testWriteRow(null, DistributionMode.NONE); + + if (parallelism > 1) { + if (partitioned) { + int files = partitionFiles("aaa") + partitionFiles("bbb") + partitionFiles("ccc"); + assertThat(files).as("Should have more than 3 files in iceberg table.").isGreaterThan(3); + } + } + } + + @TestTemplate + void testJobHashDistributionMode() { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + assertThatThrownBy(() -> testWriteRow(null, DistributionMode.RANGE)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Flink does not support 'range' write distribution mode now."); + } + + @TestTemplate + void testJobNullDistributionMode() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + testWriteRow(null, null); + + if (partitioned) { + assertThat(partitionFiles("aaa")) + .as("There should be only 1 data file in partition 'aaa'") + .isEqualTo(1); + assertThat(partitionFiles("bbb")) + .as("There should be only 1 data file in partition 'bbb'") + .isEqualTo(1); + assertThat(partitionFiles("ccc")) + .as("There should be only 1 data file in partition 'ccc'") + .isEqualTo(1); + } + } + + @TestTemplate + void testPartitionWriteMode() throws Exception { + testWriteRow(null, DistributionMode.HASH); + if (partitioned) { + assertThat(partitionFiles("aaa")) + .as("There should be only 1 data file in partition 'aaa'") + .isEqualTo(1); + assertThat(partitionFiles("bbb")) + .as("There should be only 1 data file in partition 'bbb'") + .isEqualTo(1); + assertThat(partitionFiles("ccc")) + .as("There should be only 1 data file in partition 'ccc'") + .isEqualTo(1); + } + } + + @TestTemplate + void testShuffleByPartitionWithSchema() throws Exception { + testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.HASH); + if (partitioned) { + assertThat(partitionFiles("aaa")) + .as("There should be only 1 data file in partition 'aaa'") + .isEqualTo(1); + assertThat(partitionFiles("bbb")) + .as("There should be only 1 data file in partition 'bbb'") + .isEqualTo(1); + assertThat(partitionFiles("ccc")) + .as("There should be only 1 data file in partition 'ccc'") + .isEqualTo(1); + } + } + + @TestTemplate + void testTwoSinksInDisjointedDAG() throws Exception { + Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); + + Table leftTable = + CATALOG_EXTENSION + .catalog() + .createTable( + TableIdentifier.of("left"), + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + props); + TableLoader leftTableLoader = + TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("left")); + + Table rightTable = + CATALOG_EXTENSION + .catalog() + .createTable( + TableIdentifier.of("right"), + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + props); + TableLoader rightTableLoader = + TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("right")); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + env.getConfig().disableAutoGeneratedUIDs(); + + List leftRows = createRows("left-"); + DataStream leftStream = + env.fromCollection(leftRows, ROW_TYPE_INFO) + .name("leftCustomSource") + .uid("leftCustomSource"); + IcebergSink.forRow(leftStream, SimpleDataUtil.FLINK_SCHEMA) + .table(leftTable) + .tableLoader(leftTableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .distributionMode(DistributionMode.NONE) + .uidSuffix("leftIcebergSink") + .append(); + + List rightRows = createRows("right-"); + DataStream rightStream = + env.fromCollection(rightRows, ROW_TYPE_INFO) + .name("rightCustomSource") + .uid("rightCustomSource"); + IcebergSink.forRow(rightStream, SimpleDataUtil.FLINK_SCHEMA) + .table(rightTable) + .tableLoader(rightTableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .distributionMode(DistributionMode.HASH) + .uidSuffix("rightIcebergSink") + .setSnapshotProperty("flink.test", TestIcebergSink.class.getName()) + .snapshotProperties(Collections.singletonMap("direction", "rightTable")) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(leftTable, convertToRowData(leftRows)); + SimpleDataUtil.assertTableRows(rightTable, convertToRowData(rightRows)); + + leftTable.refresh(); + + assertThat(leftTable.currentSnapshot().summary().get("flink.test")).isNull(); + assertThat(leftTable.currentSnapshot().summary().get("direction")).isNull(); + + assertThat(rightTable.currentSnapshot().summary().get("flink.test")) + .isEqualTo(TestIcebergSink.class.getName()); + assertThat(rightTable.currentSnapshot().summary().get("direction")).isEqualTo("rightTable"); + } + + @TestTemplate + void testOverrideWriteConfigWithUnknownDistributionMode() { + Map newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), "UNRECOGNIZED"); + + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .setAll(newProps) + .append(); + + assertThatThrownBy(() -> env.execute("Test Iceberg DataStream")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid distribution mode: UNRECOGNIZED"); + } + + @TestTemplate + void testOverrideWriteConfigWithUnknownFileFormat() { + Map newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.WRITE_FORMAT.key(), "UNRECOGNIZED"); + + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + Builder builder = + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .setAll(newProps) + .uidSuffix("ingestion"); + assertThatThrownBy(builder::append) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid file format: UNRECOGNIZED"); + } + + @TestTemplate + void testWriteRowWithTableRefreshInterval() throws Exception { + List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) + .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); + + Configuration flinkConf = new Configuration(); + flinkConf.setString(FlinkWriteOptions.TABLE_REFRESH_INTERVAL.key(), "100ms"); + + IcebergSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .flinkConf(flinkConf) + .writeParallelism(parallelism) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream"); + + // Assert the iceberg table's records. + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } + + @TestTemplate + void testOperatorsUidNameNoUidSuffix() throws Exception { + List rows = createRows(""); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO).uid("mySourceId"); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .distributionMode(DistributionMode.HASH) + .append(); + + Transformation firstTransformation = env.getTransformations().get(0); + Transformation secondTransformation = env.getTransformations().get(1); + assertThat(firstTransformation.getUid()).isEqualTo("Sink pre-writer mapper: hadoop.default.t"); + assertThat(firstTransformation.getName()).isEqualTo("Sink pre-writer mapper: hadoop.default.t"); + assertThat(secondTransformation.getUid()).isEqualTo("hadoop.default.t"); + assertThat(secondTransformation.getName()).isEqualTo("hadoop.default.t"); + } + + @TestTemplate + void testOperatorsUidNameWitUidSuffix() throws Exception { + List rows = createRows(""); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO).uid("mySourceId"); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .distributionMode(DistributionMode.HASH) + .uidSuffix("data-ingestion") + .append(); + + Transformation firstTransformation = env.getTransformations().get(0); + Transformation secondTransformation = env.getTransformations().get(1); + assertThat(firstTransformation.getUid()).isEqualTo("Sink pre-writer mapper: data-ingestion"); + assertThat(firstTransformation.getName()).isEqualTo("Sink pre-writer mapper: data-ingestion"); + assertThat(secondTransformation.getUid()).isEqualTo("data-ingestion"); + assertThat(secondTransformation.getName()).isEqualTo("data-ingestion"); + } + + private void testWriteRow(TableSchema tableSchema, DistributionMode distributionMode) + throws Exception { + List rows = createRows(""); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO).uid("mySourceId"); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .writeParallelism(parallelism) + .distributionMode(distributionMode) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java new file mode 100644 index 000000000000..a8c8892af11c --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.util.List; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergSinkBranch extends TestFlinkIcebergSinkBase { + + @Parameter(index = 0) + private String branch; + + @Parameters(name = "branch = {0}") + public static Object[][] parameters() { + return new Object[][] {new Object[] {"main"}, new Object[] {"testBranch"}}; + } + + @BeforeEach + public void before() throws IOException { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + FileFormat.AVRO.name(), + TableProperties.FORMAT_VERSION, + "1")); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100); + + tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testWriteRowWithTableSchema() throws Exception { + testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); + verifyOtherBranchUnmodified(); + } + + private void testWriteRow(TableSchema tableSchema, DistributionMode distributionMode) + throws Exception { + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .toBranch(branch) + .distributionMode(distributionMode) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(table, convertToRowData(rows), branch); + SimpleDataUtil.assertTableRows( + table, + ImmutableList.of(), + branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH); + + verifyOtherBranchUnmodified(); + } + + private void verifyOtherBranchUnmodified() { + String otherBranch = + branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; + if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { + assertThat(table.currentSnapshot()).isNull(); + } + + assertThat(table.snapshot(otherBranch)).isNull(); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java new file mode 100644 index 000000000000..007d5f314946 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java @@ -0,0 +1,267 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.apache.iceberg.flink.TestFixtures.DATABASE; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.util.List; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.BoundedTestSource; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; + +@ExtendWith(ParameterizedTestExtension.class) +@Timeout(value = 60) +public class TestIcebergSinkV2 extends TestFlinkIcebergSinkV2Base { + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); + + @BeforeEach + public void setupTable() { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + format.name(), + TableProperties.FORMAT_VERSION, + String.valueOf(FORMAT_V2))); + + table + .updateProperties() + .set(TableProperties.DEFAULT_FILE_FORMAT, format.name()) + .set(TableProperties.WRITE_DISTRIBUTION_MODE, writeDistributionMode) + .commit(); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100L) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testCheckAndGetEqualityFieldIds() { + table + .updateSchema() + .allowIncompatibleChanges() + .addRequiredColumn("type", Types.StringType.get()) + .setIdentifierFields("type") + .commit(); + + DataStream dataStream = + env.addSource(new BoundedTestSource<>(ImmutableList.of()), ROW_TYPE_INFO); + IcebergSink.Builder builder = + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA).table(table); + + // Use user-provided equality field column as equality field id list + builder.equalityFieldColumns(Lists.newArrayList("id")); + assertThat(SinkUtil.checkAndGetEqualityFieldIds(table, Lists.newArrayList("id"))) + .containsExactlyInAnyOrder(table.schema().findField("id").fieldId()); + } + + @TestTemplate + public void testChangeLogOnIdKey() throws Exception { + testChangeLogOnIdKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertOnlyDeletesOnDataKey() throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of(row("+I", 1, "aaa")), + ImmutableList.of(row("-D", 1, "aaa"), row("-D", 2, "bbb"))); + + List> expectedRecords = + ImmutableList.of(ImmutableList.of(record(1, "aaa")), ImmutableList.of()); + + testChangeLogs( + ImmutableList.of("data"), + row -> row.getField(ROW_DATA_POS), + true, + elementsPerCheckpoint, + expectedRecords, + SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testChangeLogOnDataKey() throws Exception { + testChangeLogOnDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testChangeLogOnIdDataKey() throws Exception { + testChangeLogOnIdDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testChangeLogOnSameKey() throws Exception { + testChangeLogOnSameKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertModeCheck() throws Exception { + DataStream dataStream = + env.addSource(new BoundedTestSource<>(ImmutableList.of()), ROW_TYPE_INFO); + IcebergSink.Builder builder = + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .upsert(true); + + assertThatThrownBy( + () -> + builder + .equalityFieldColumns(ImmutableList.of("id", "data")) + .overwrite(true) + .append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); + + assertThatThrownBy( + () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + } + + @TestTemplate + public void testUpsertOnIdKey() throws Exception { + testUpsertOnIdKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertOnDataKey() throws Exception { + testUpsertOnDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertOnIdDataKey() throws Exception { + testUpsertOnIdDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testDeleteStats() throws Exception { + assumeThat(format).isNotEqualTo(FileFormat.AVRO); + + List> elementsPerCheckpoint = + ImmutableList.of( + // Checkpoint #1 + ImmutableList.of(row("+I", 1, "aaa"), row("-D", 1, "aaa"), row("+I", 1, "aaa"))); + + List> expectedRecords = ImmutableList.of(ImmutableList.of(record(1, "aaa"))); + + testChangeLogs( + ImmutableList.of("id", "data"), + row -> Row.of(row.getField(ROW_ID_POS), row.getField(ROW_DATA_POS)), + false, + elementsPerCheckpoint, + expectedRecords, + "main"); + + DeleteFile deleteFile = table.currentSnapshot().addedDeleteFiles(table.io()).iterator().next(); + String fromStat = + new String( + deleteFile.lowerBounds().get(MetadataColumns.DELETE_FILE_PATH.fieldId()).array()); + DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + assumeThat(fromStat).isEqualTo(dataFile.path().toString()); + } + + protected void testChangeLogs( + List equalityFieldColumns, + KeySelector keySelector, + boolean insertAsUpsert, + List> elementsPerCheckpoint, + List> expectedRecordsPerCheckpoint, + String branch) + throws Exception { + DataStream dataStream = + env.addSource(new BoundedTestSource<>(elementsPerCheckpoint), ROW_TYPE_INFO); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .equalityFieldColumns(equalityFieldColumns) + .upsert(insertAsUpsert) + .toBranch(branch) + .uidSuffix("sink") + .append(); + + // Execute the program. + env.execute("Test Iceberg Change-Log DataStream."); + + table.refresh(); + List snapshots = findValidSnapshots(); + int expectedSnapshotNum = expectedRecordsPerCheckpoint.size(); + assertThat(snapshots).hasSize(expectedSnapshotNum); + + for (int i = 0; i < expectedSnapshotNum; i++) { + long snapshotId = snapshots.get(i).snapshotId(); + List expectedRecords = expectedRecordsPerCheckpoint.get(i); + assertThat(actualRowSet(snapshotId, "*")) + .as("Should have the expected records for the checkpoint#" + i) + .isEqualTo(expectedRowSet(expectedRecords.toArray(new Record[0]))); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2Branch.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2Branch.java new file mode 100644 index 000000000000..4896f7f48c17 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2Branch.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Branch { + + @BeforeEach + @Override + public void before() throws IOException { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + FileFormat.AVRO.name(), + TableProperties.FORMAT_VERSION, + "2")); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100); + + tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testChangeLogOnIdKey() throws Exception { + testChangeLogOnIdKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testChangeLogOnDataKey() throws Exception { + testChangeLogOnDataKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testChangeLogOnIdDataKey() throws Exception { + testChangeLogOnIdDataKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testUpsertOnIdKey() throws Exception { + testUpsertOnIdKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testUpsertOnDataKey() throws Exception { + testUpsertOnDataKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testUpsertOnIdDataKey() throws Exception { + testUpsertOnIdDataKey(branch); + verifyOtherBranchUnmodified(); + } + + private void verifyOtherBranchUnmodified() { + String otherBranch = + branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; + if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { + assertThat(table.currentSnapshot()); + } + + assertThat(table.snapshot(otherBranch)).isNull(); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java index 50283f7ad215..e13721a9f170 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java @@ -28,6 +28,7 @@ import java.util.Locale; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; import org.apache.flink.streaming.api.operators.BoundedOneInput; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.table.api.DataTypes; @@ -102,7 +103,7 @@ public void before() throws IOException { @TestTemplate public void testWritingTable() throws Exception { long checkpointId = 1L; - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { // The first checkpoint testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); @@ -111,7 +112,8 @@ public void testWritingTable() throws Exception { testHarness.prepareSnapshotPreBarrier(checkpointId); int expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); @@ -123,7 +125,8 @@ public void testWritingTable() throws Exception { testHarness.prepareSnapshotPreBarrier(checkpointId); expectedDataFiles = partitioned ? 4 : 2; - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); @@ -148,14 +151,15 @@ public void testWritingTable() throws Exception { public void testSnapshotTwice() throws Exception { long checkpointId = 1; long timestamp = 1; - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), timestamp++); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), timestamp); testHarness.prepareSnapshotPreBarrier(checkpointId++); int expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); @@ -163,7 +167,10 @@ public void testSnapshotTwice() throws Exception { for (int i = 0; i < 5; i++) { testHarness.prepareSnapshotPreBarrier(checkpointId++); - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + result = + WriteResult.builder() + .addAll(getWriteResults(testHarness.extractOutputValues())) + .build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); } @@ -172,14 +179,14 @@ public void testSnapshotTwice() throws Exception { @TestTemplate public void testTableWithoutSnapshot() throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { assertThat(testHarness.extractOutputValues()).isEmpty(); } // Even if we closed the iceberg stream writer, there's no orphan data file. assertThat(scanDataFiles()).isEmpty(); - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); // Still not emit the data file yet, because there is no checkpoint. @@ -212,7 +219,7 @@ private Set scanDataFiles() throws IOException { @TestTemplate public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); @@ -221,13 +228,15 @@ public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); int expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); // Datafiles should not be sent again assertThat(result.dataFiles()).hasSize(expectedDataFiles); @@ -236,7 +245,7 @@ public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { @TestTemplate public void testBoundedStreamTriggeredEndInputBeforeTriggeringCheckpoint() throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); @@ -244,13 +253,15 @@ public void testBoundedStreamTriggeredEndInputBeforeTriggeringCheckpoint() throw testHarness.endInput(); int expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); testHarness.prepareSnapshotPreBarrier(1L); - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); // It should be ensured that after endInput is triggered, when prepareSnapshotPreBarrier // is triggered, write should only send WriteResult once @@ -275,7 +286,7 @@ public void testTableWithTargetFileSize() throws Exception { } } - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { for (RowData row : rows) { testHarness.processElement(row, 1); @@ -283,7 +294,8 @@ public void testTableWithTargetFileSize() throws Exception { // snapshot the operator. testHarness.prepareSnapshotPreBarrier(1); - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(8); @@ -346,13 +358,14 @@ public void testPromotedFlinkDataType() throws Exception { record.copy(ImmutableMap.of("tinyint", 2, "smallint", 0, "int", 102)), record.copy(ImmutableMap.of("tinyint", 3, "smallint", 32767, "int", 103))); - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter(icebergTable, flinkSchema)) { for (RowData row : rows) { testHarness.processElement(row, 1); } testHarness.prepareSnapshotPreBarrier(1); - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(partitioned ? 3 : 1); @@ -365,12 +378,18 @@ public void testPromotedFlinkDataType() throws Exception { SimpleDataUtil.assertTableRecords(location, expected); } - private OneInputStreamOperatorTestHarness createIcebergStreamWriter() + private static List getWriteResults(List flinkWriteResults) { + return flinkWriteResults.stream() + .map(FlinkWriteResult::writeResult) + .collect(Collectors.toList()); + } + + private OneInputStreamOperatorTestHarness createIcebergStreamWriter() throws Exception { return createIcebergStreamWriter(table, SimpleDataUtil.FLINK_SCHEMA); } - private OneInputStreamOperatorTestHarness createIcebergStreamWriter( + private OneInputStreamOperatorTestHarness createIcebergStreamWriter( Table icebergTable, TableSchema flinkSchema) throws Exception { RowType flinkRowType = FlinkSink.toFlinkRowType(icebergTable.schema(), flinkSchema); FlinkWriteConf flinkWriteConfig = @@ -379,7 +398,7 @@ private OneInputStreamOperatorTestHarness createIcebergStr IcebergStreamWriter streamWriter = FlinkSink.createStreamWriter(() -> icebergTable, flinkWriteConfig, flinkRowType, null); - OneInputStreamOperatorTestHarness harness = + OneInputStreamOperatorTestHarness harness = new OneInputStreamOperatorTestHarness<>(streamWriter, 1, 1, 0); harness.setup(); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java index d5a0bebc74e7..c95a9955b104 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java @@ -380,7 +380,9 @@ private static Map>> runPartitioner( return partitionResults; } - /** @param expectedAssignmentInfo excluding closing cost */ + /** + * @param expectedAssignmentInfo excluding closing cost + */ private void validatePartitionResults( Map> expectedAssignmentInfo, Map>> partitionResults, diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java index e1162c3225b1..2dc5bc5c658e 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java @@ -33,10 +33,12 @@ import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; /** Use the FlinkSource */ public class TestFlinkSourceSql extends TestSqlBase { + @BeforeEach @Override public void before() throws IOException { SqlHelpers.sql( diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java index b7447d15c05a..db8647f054ae 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java @@ -24,8 +24,6 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -130,11 +128,8 @@ protected List run( sourceBuilder.properties(options); DataStream stream = - env.fromSource( - sourceBuilder.build(), - WatermarkStrategy.noWatermarks(), - "testBasicRead", - TypeInformation.of(RowData.class)) + sourceBuilder + .buildStream(env) .map( new RowDataToRowMapper( FlinkSchemaUtil.convert( diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java index 7bfed00a9eb4..4e649d15b1ce 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java @@ -52,6 +52,7 @@ import org.apache.iceberg.flink.data.RowDataToRowMapper; import org.apache.iceberg.flink.sink.AvroGenericRecordToRowDataMapper; import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; +import org.apache.iceberg.flink.source.reader.AvroGenericRecordConverter; import org.apache.iceberg.flink.source.reader.AvroGenericRecordReaderFunction; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; @@ -69,12 +70,13 @@ public class TestIcebergSourceBoundedGenericRecord { private static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); - @Parameters(name = "format={0}, parallelism = {1}") + @Parameters(name = "format={0}, parallelism = {1}, useConverter = {2}") public static Object[][] parameters() { return new Object[][] { - {FileFormat.AVRO, 2}, - {FileFormat.PARQUET, 2}, - {FileFormat.ORC, 2} + {FileFormat.AVRO, 2, true}, + {FileFormat.PARQUET, 2, true}, + {FileFormat.PARQUET, 2, false}, + {FileFormat.ORC, 2, true} }; } @@ -84,6 +86,9 @@ public static Object[][] parameters() { @Parameter(index = 1) private int parallelism; + @Parameter(index = 2) + private boolean useConverter; + @TestTemplate public void testUnpartitionedTable() throws Exception { Table table = @@ -147,24 +152,15 @@ private List run( table = tableLoader.loadTable(); } - AvroGenericRecordReaderFunction readerFunction = - new AvroGenericRecordReaderFunction( - TestFixtures.TABLE_IDENTIFIER.name(), - new Configuration(), - table.schema(), - null, - null, - false, - table.io(), - table.encryption(), - filters); + Schema readSchema = projectedSchema != null ? projectedSchema : table.schema(); + IcebergSource.Builder sourceBuilder; + if (useConverter) { + sourceBuilder = createSourceBuilderWithConverter(table, readSchema, config); + } else { + sourceBuilder = + createSourceBuilderWithReaderFunction(table, projectedSchema, filters, config); + } - IcebergSource.Builder sourceBuilder = - IcebergSource.builder() - .tableLoader(CATALOG_EXTENSION.tableLoader()) - .readerFunction(readerFunction) - .assignerFactory(new SimpleSplitAssignerFactory()) - .flinkConfig(config); if (projectedSchema != null) { sourceBuilder.project(projectedSchema); } @@ -172,7 +168,6 @@ private List run( sourceBuilder.filters(filters); sourceBuilder.setAll(options); - Schema readSchema = projectedSchema != null ? projectedSchema : table.schema(); RowType rowType = FlinkSchemaUtil.convert(readSchema); org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(readSchema, TestFixtures.TABLE_IDENTIFIER.name()); @@ -193,4 +188,35 @@ private List run( return Lists.newArrayList(iter); } } + + private IcebergSource.Builder createSourceBuilderWithReaderFunction( + Table table, Schema projected, List filters, Configuration config) { + AvroGenericRecordReaderFunction readerFunction = + new AvroGenericRecordReaderFunction( + TestFixtures.TABLE_IDENTIFIER.name(), + new Configuration(), + table.schema(), + projected, + null, + false, + table.io(), + table.encryption(), + filters); + + return IcebergSource.builder() + .tableLoader(CATALOG_EXTENSION.tableLoader()) + .readerFunction(readerFunction) + .assignerFactory(new SimpleSplitAssignerFactory()) + .flinkConfig(config); + } + + private IcebergSource.Builder createSourceBuilderWithConverter( + Table table, Schema readSchema, Configuration config) { + AvroGenericRecordConverter converter = + AvroGenericRecordConverter.fromIcebergSchema(readSchema, table.name()); + return IcebergSource.forOutputType(converter) + .tableLoader(CATALOG_EXTENSION.tableLoader()) + .assignerFactory(new SimpleSplitAssignerFactory()) + .flinkConfig(config); + } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java index 0f41c5af4c95..d3713e296014 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java @@ -37,7 +37,7 @@ public class TestIcebergSourceBoundedSql extends TestIcebergSourceBounded { @BeforeEach public void before() throws IOException { Configuration tableConf = getTableEnv().getConfig().getConfiguration(); - tableConf.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), true); + tableConf.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); SqlHelpers.sql( getTableEnv(), "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java new file mode 100644 index 000000000000..2908cb927269 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java @@ -0,0 +1,181 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source; + +import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.nio.file.Path; +import java.util.List; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; +import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.testutils.InternalMiniClusterExtension; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.data.RowDataToRowMapper; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +public class TestIcebergSourceInferParallelism { + private static final int NUM_TMS = 2; + private static final int SLOTS_PER_TM = 2; + private static final int PARALLELISM = NUM_TMS * SLOTS_PER_TM; + private static final int MAX_INFERRED_PARALLELISM = 3; + + @RegisterExtension + private static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(NUM_TMS) + .setNumberSlotsPerTaskManager(SLOTS_PER_TM) + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) + .build()); + + @RegisterExtension + protected static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + @TempDir private Path tmpDir; + + private Table table; + private GenericAppenderHelper dataAppender; + + @BeforeEach + public void before() throws IOException { + this.table = + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + this.dataAppender = new GenericAppenderHelper(table, FileFormat.PARQUET, tmpDir); + } + + @AfterEach + public void after() { + CATALOG_EXTENSION.catalog().dropTable(TestFixtures.TABLE_IDENTIFIER); + } + + @Test + public void testEmptyTable() throws Exception { + // Inferred parallelism should be at least 1 even if table is empty + test(1, 0); + } + + @Test + public void testTableWithFilesLessThanMaxInferredParallelism() throws Exception { + // Append files to the table + for (int i = 0; i < 2; ++i) { + List batch = RandomGenericData.generate(table.schema(), 1, 0); + dataAppender.appendToTable(batch); + } + + // Inferred parallelism should equal to 2 splits + test(2, 2); + } + + @Test + public void testTableWithFilesMoreThanMaxInferredParallelism() throws Exception { + // Append files to the table + for (int i = 0; i < MAX_INFERRED_PARALLELISM + 1; ++i) { + List batch = RandomGenericData.generate(table.schema(), 1, 0); + dataAppender.appendToTable(batch); + } + + // Inferred parallelism should be capped by the MAX_INFERRED_PARALLELISM + test(MAX_INFERRED_PARALLELISM, MAX_INFERRED_PARALLELISM + 1); + } + + private void test(int expectedParallelism, int expectedRecords) throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(PARALLELISM); + + Configuration config = new Configuration(); + config.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, true); + config.set( + FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX, + MAX_INFERRED_PARALLELISM); + + DataStream dataStream = + IcebergSource.forRowData() + .tableLoader(CATALOG_EXTENSION.tableLoader()) + .table(table) + .flinkConfig(config) + // force one file per split + .splitSize(1L) + .buildStream(env) + .map(new RowDataToRowMapper(FlinkSchemaUtil.convert(table.schema()))); + + DataStream.Collector collector = new DataStream.Collector<>(); + dataStream.collectAsync(collector); + JobClient jobClient = env.executeAsync(); + try (CloseableIterator iterator = collector.getOutput()) { + List result = Lists.newArrayList(); + while (iterator.hasNext()) { + result.add(iterator.next()); + } + + assertThat(result).hasSize(expectedRecords); + verifySourceParallelism( + expectedParallelism, miniCluster().getExecutionGraph(jobClient.getJobID()).get()); + } + } + + /** + * Borrowed this approach from Flink {@code FileSourceTextLinesITCase} to get source parallelism + * from execution graph. + */ + private static void verifySourceParallelism( + int expectedParallelism, AccessExecutionGraph executionGraph) { + AccessExecutionJobVertex sourceVertex = + executionGraph.getVerticesTopologically().iterator().next(); + assertThat(sourceVertex.getParallelism()).isEqualTo(expectedParallelism); + } + + /** + * Use reflection to get {@code InternalMiniClusterExtension} and {@code MiniCluster} to get + * execution graph and source parallelism. Haven't find other way via public APIS. + */ + private static MiniCluster miniCluster() throws Exception { + Field privateField = + MiniClusterExtension.class.getDeclaredField("internalMiniClusterExtension"); + privateField.setAccessible(true); + InternalMiniClusterExtension internalExtension = + (InternalMiniClusterExtension) privateField.get(MINI_CLUSTER_EXTENSION); + return internalExtension.getMiniCluster(); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java index 75f0a785a8c5..66bdeee1d407 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java @@ -40,6 +40,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; /** Use the IcebergSource (FLIP-27) */ @@ -49,11 +50,17 @@ public class TestIcebergSourceSql extends TestSqlBase { required(1, "t1", Types.TimestampType.withoutZone()), required(2, "t2", Types.LongType.get())); + @BeforeEach @Override public void before() throws IOException { TableEnvironment tableEnvironment = getTableEnv(); Configuration tableConf = tableEnvironment.getConfig().getConfiguration(); - tableConf.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), true); + tableConf.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); + // Disable inferring parallelism to avoid interfering watermark tests + // that check split assignment is ordered by the watermark column. + // The tests assumes default parallelism of 1 with single reader task + // in order to check the order of read records. + tableConf.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); tableEnvironment.getConfig().set("table.exec.resource.default-parallelism", "1"); SqlHelpers.sql( diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java index 41b023b93617..05a08c24d8d0 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java @@ -27,6 +27,7 @@ import java.util.List; import java.util.stream.Collectors; import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.common.TaskInfo; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.configuration.BatchExecutionOptions; @@ -48,8 +49,14 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.extension.RegisterExtension; +/** + * There is a infinite sleep in the test. Add a timeout to the test to avoid stuck situation in case + * anything goes wrong unexpectedly. + */ +@Timeout(value = 60) public class TestIcebergSpeculativeExecutionSupport extends TestBase { private static final int NUM_TASK_MANAGERS = 1; private static final int NUM_TASK_SLOTS = 3; @@ -144,9 +151,9 @@ public void testSpeculativeExecution() throws Exception { private static class TestingMap extends RichMapFunction { @Override public Row map(Row row) throws Exception { - // Put the subtasks with the first attempt to sleep to trigger speculative - // execution - if (getRuntimeContext().getTaskInfo().getAttemptNumber() <= 0) { + // Simulate slow subtask 0 with attempt 0 + TaskInfo taskInfo = getRuntimeContext().getTaskInfo(); + if (taskInfo.getIndexOfThisSubtask() == 0 && taskInfo.getAttemptNumber() <= 0) { Thread.sleep(Integer.MAX_VALUE); } @@ -169,6 +176,7 @@ private static Configuration configure() { // Use FLIP-27 source configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); + configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); // for speculative execution configuration.set(BatchExecutionOptions.SPECULATIVE_ENABLED, true); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java index 9cf953342a18..9cdcb72c12cf 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java @@ -217,27 +217,27 @@ public void testPrimitiveColumns() throws Exception { Row binaryCol = Row.of( - 52L, + 55L, 4L, 2L, null, Base64.getDecoder().decode("1111"), Base64.getDecoder().decode("2222")); - Row booleanCol = Row.of(32L, 4L, 0L, null, false, true); - Row decimalCol = Row.of(85L, 4L, 1L, null, new BigDecimal("1.00"), new BigDecimal("2.00")); - Row doubleCol = Row.of(85L, 4L, 0L, 1L, 1.0D, 2.0D); + Row booleanCol = Row.of(36L, 4L, 0L, null, false, true); + Row decimalCol = Row.of(91L, 4L, 1L, null, new BigDecimal("1.00"), new BigDecimal("2.00")); + Row doubleCol = Row.of(91L, 4L, 0L, 1L, 1.0D, 2.0D); Row fixedCol = Row.of( - 44L, + 47L, 4L, 2L, null, Base64.getDecoder().decode("1111"), Base64.getDecoder().decode("2222")); - Row floatCol = Row.of(71L, 4L, 0L, 2L, 0f, 0f); - Row intCol = Row.of(71L, 4L, 0L, null, 1, 2); - Row longCol = Row.of(79L, 4L, 0L, null, 1L, 2L); - Row stringCol = Row.of(79L, 4L, 0L, null, "1", "2"); + Row floatCol = Row.of(77L, 4L, 0L, 2L, 0f, 0f); + Row intCol = Row.of(77L, 4L, 0L, null, 1, 2); + Row longCol = Row.of(85L, 4L, 0L, null, 1L, 2L); + Row stringCol = Row.of(85L, 4L, 0L, null, "1", "2"); List expected = Lists.newArrayList( @@ -289,8 +289,8 @@ public void testSelectNestedValues() throws Exception { public void testNestedValues() throws Exception { createNestedTable(); - Row leafDoubleCol = Row.of(46L, 3L, 1L, 1L, 0.0D, 0.0D); - Row leafLongCol = Row.of(54L, 3L, 1L, null, 0L, 1L); + Row leafDoubleCol = Row.of(50L, 3L, 1L, 1L, 0.0D, 0.0D); + Row leafLongCol = Row.of(57L, 3L, 1L, null, 0L, 1L); Row metrics = Row.of(Row.of(leafDoubleCol, leafLongCol)); TestHelpers.assertRows( diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java index 0690b456e033..5767fa02c822 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java @@ -80,7 +80,9 @@ private void appendTwoSnapshots() throws IOException { snapshot2 = TABLE_RESOURCE.table().currentSnapshot(); } - /** @return the last enumerated snapshot id */ + /** + * @return the last enumerated snapshot id + */ private CycleResult verifyOneCycle( ContinuousSplitPlannerImpl splitPlanner, IcebergEnumeratorPosition lastPosition) throws Exception { diff --git a/flink/v1.19/flink/src/test/resources/META-INF/services/org.apache.flink.metrics.reporter.MetricReporterFactory b/flink/v1.19/flink/src/test/resources/META-INF/services/org.apache.flink.metrics.reporter.MetricReporterFactory new file mode 100644 index 000000000000..952255a52b7c --- /dev/null +++ b/flink/v1.19/flink/src/test/resources/META-INF/services/org.apache.flink.metrics.reporter.MetricReporterFactory @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +org.apache.iceberg.flink.maintenance.operator.MetricsReporterFactoryForTests diff --git a/flink/v1.20/build.gradle b/flink/v1.20/build.gradle index f2e1fb51a1f4..4a1bae660bdb 100644 --- a/flink/v1.20/build.gradle +++ b/flink/v1.20/build.gradle @@ -119,6 +119,7 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { testImplementation libs.awaitility testImplementation libs.assertj.core + testImplementation libs.sqlite.jdbc } test { @@ -127,7 +128,7 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { } project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") { - apply plugin: 'io.github.goooler.shadow' + apply plugin: 'com.gradleup.shadow' tasks.jar.dependsOn tasks.shadowJar diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java index 7167859e600c..d5eea6706b39 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java @@ -53,6 +53,10 @@ public LongConfParser longConf() { return new LongConfParser(); } + public DoubleConfParser doubleConf() { + return new DoubleConfParser(); + } + public > EnumConfParser enumConfParser(Class enumClass) { return new EnumConfParser<>(enumClass); } @@ -135,6 +139,29 @@ public Long parseOptional() { } } + class DoubleConfParser extends ConfParser { + private Double defaultValue; + + @Override + protected DoubleConfParser self() { + return this; + } + + public DoubleConfParser defaultValue(double value) { + this.defaultValue = value; + return self(); + } + + public double parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(Double::parseDouble, defaultValue); + } + + public Double parseOptional() { + return parse(Double::parseDouble, null); + } + } + class StringConfParser extends ConfParser { private String defaultValue; diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java index 7c7afd24ed8e..099925101571 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java @@ -88,7 +88,7 @@ private FlinkConfigOptions() {} public static final ConfigOption TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE = ConfigOptions.key("table.exec.iceberg.use-flip27-source") .booleanType() - .defaultValue(false) + .defaultValue(true) .withDescription("Use the FLIP-27 based Iceberg source implementation."); public static final ConfigOption TABLE_EXEC_SPLIT_ASSIGNER_TYPE = diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java index 4790dc85bf28..afc09a719f77 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java @@ -58,7 +58,9 @@ public class FlinkSchemaUtil { private FlinkSchemaUtil() {} - /** @deprecated Use {@link #convert(ResolvedSchema)} instead. */ + /** + * @deprecated Use {@link #convert(ResolvedSchema)} instead. + */ @Deprecated public static Schema convert(TableSchema schema) { LogicalType schemaType = schema.toRowDataType().getLogicalType(); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java index ca7b1120bc81..a31902d49a8b 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java @@ -26,6 +26,7 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.sink.shuffle.StatisticsType; /** * A class for common Iceberg configs for Flink writes. @@ -167,6 +168,26 @@ public DistributionMode distributionMode() { return DistributionMode.fromName(modeName); } + public StatisticsType rangeDistributionStatisticsType() { + String name = + confParser + .stringConf() + .option(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE.key()) + .flinkConfig(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE) + .defaultValue(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE.defaultValue()) + .parse(); + return StatisticsType.valueOf(name); + } + + public double rangeDistributionSortKeyBaseWeight() { + return confParser + .doubleConf() + .option(FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT.key()) + .flinkConfig(FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT) + .defaultValue(FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT.defaultValue()) + .parse(); + } + public int workerPoolSize() { return confParser .intConf() diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java index df73f2e09cac..c35286774874 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java @@ -23,6 +23,7 @@ import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.flink.sink.shuffle.StatisticsType; /** Flink sink write options */ public class FlinkWriteOptions { @@ -60,6 +61,19 @@ private FlinkWriteOptions() {} public static final ConfigOption DISTRIBUTION_MODE = ConfigOptions.key("distribution-mode").stringType().noDefaultValue(); + public static final ConfigOption RANGE_DISTRIBUTION_STATISTICS_TYPE = + ConfigOptions.key("range-distribution-statistics-type") + .stringType() + .defaultValue(StatisticsType.Auto.name()) + .withDescription("Type of statistics collection: Auto, Map, Sketch"); + + public static final ConfigOption RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT = + ConfigOptions.key("range-distribution-sort-key-base-weight") + .doubleType() + .defaultValue(0.0d) + .withDescription( + "Base weight for every sort key relative to target weight per writer task"); + // Branch to write to public static final ConfigOption BRANCH = ConfigOptions.key("branch").stringType().defaultValue(SnapshotRef.MAIN_BRANCH); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java index 1019285018d0..34576a1e5c0b 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java @@ -237,10 +237,10 @@ public MapData getMap(int pos) { @Override public RowData getRow(int pos, int numFields) { - return isNullAt(pos) ? null : getStructRowData(pos, numFields); + return isNullAt(pos) ? null : getStructRowData(pos); } - private StructRowData getStructRowData(int pos, int numFields) { + private StructRowData getStructRowData(int pos) { return new StructRowData( type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class)); } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/JdbcLockFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/JdbcLockFactory.java new file mode 100644 index 000000000000..085fbfecd270 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/JdbcLockFactory.java @@ -0,0 +1,323 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import java.io.IOException; +import java.sql.DatabaseMetaData; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.SQLNonTransientConnectionException; +import java.sql.SQLTimeoutException; +import java.sql.SQLTransientConnectionException; +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.jdbc.JdbcClientPool; +import org.apache.iceberg.jdbc.UncheckedInterruptedException; +import org.apache.iceberg.jdbc.UncheckedSQLException; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.PropertyUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * JDBC table backed implementation of the {@link + * org.apache.iceberg.flink.maintenance.operator.TriggerLockFactory}. + */ +public class JdbcLockFactory implements TriggerLockFactory { + private static final Logger LOG = LoggerFactory.getLogger(JdbcLockFactory.class); + + @VisibleForTesting + static final String INIT_LOCK_TABLES_PROPERTY = "flink-maintenance.lock.jdbc.init-lock-tables"; + + private static final String LOCK_TABLE_NAME = "flink_maintenance_lock"; + private static final int LOCK_ID_MAX_LENGTH = 100; + private static final String CREATE_LOCK_TABLE_SQL = + String.format( + "CREATE TABLE %s " + + "(LOCK_TYPE CHAR(1) NOT NULL, " + + "LOCK_ID VARCHAR(%s) NOT NULL, " + + "INSTANCE_ID CHAR(36) NOT NULL, PRIMARY KEY (LOCK_TYPE, LOCK_ID))", + LOCK_TABLE_NAME, LOCK_ID_MAX_LENGTH); + + private static final String CREATE_LOCK_SQL = + String.format( + "INSERT INTO %s (LOCK_TYPE, LOCK_ID, INSTANCE_ID) VALUES (?, ?, ?)", LOCK_TABLE_NAME); + private static final String GET_LOCK_SQL = + String.format("SELECT INSTANCE_ID FROM %s WHERE LOCK_TYPE=? AND LOCK_ID=?", LOCK_TABLE_NAME); + private static final String DELETE_LOCK_SQL = + String.format( + "DELETE FROM %s WHERE LOCK_TYPE=? AND LOCK_ID=? AND INSTANCE_ID=?", LOCK_TABLE_NAME); + + private final String uri; + private final String lockId; + private final Map properties; + private transient JdbcClientPool pool; + + /** + * Creates a new {@link TriggerLockFactory}. The lockId should be unique between the users of the + * same uri. + * + * @param uri of the jdbc connection + * @param lockId which should indentify the job and the table + * @param properties used for creating the jdbc connection pool + */ + public JdbcLockFactory(String uri, String lockId, Map properties) { + Preconditions.checkNotNull(uri, "JDBC connection URI is required"); + Preconditions.checkNotNull(properties, "Properties map is required"); + Preconditions.checkArgument( + lockId.length() < LOCK_ID_MAX_LENGTH, + "Invalid prefix length: lockId should be shorter than %s", + LOCK_ID_MAX_LENGTH); + this.uri = uri; + this.lockId = lockId; + this.properties = properties; + } + + @Override + public void open() { + this.pool = new JdbcClientPool(1, uri, properties); + + if (PropertyUtil.propertyAsBoolean(properties, INIT_LOCK_TABLES_PROPERTY, false)) { + initializeLockTables(); + } + } + + /** Only used in testing to share the jdbc pool */ + @VisibleForTesting + void open(JdbcLockFactory other) { + this.pool = other.pool; + } + + @Override + public Lock createLock() { + return new JdbcLock(pool, lockId, Type.MAINTENANCE); + } + + @Override + public Lock createRecoveryLock() { + return new JdbcLock(pool, lockId, Type.RECOVERY); + } + + @Override + public void close() throws IOException { + pool.close(); + } + + private void initializeLockTables() { + LOG.debug("Creating database tables (if missing) to store table maintenance locks"); + try { + pool.run( + conn -> { + DatabaseMetaData dbMeta = conn.getMetaData(); + ResultSet tableExists = + dbMeta.getTables( + null /* catalog name */, + null /* schemaPattern */, + LOCK_TABLE_NAME /* tableNamePattern */, + null /* types */); + if (tableExists.next()) { + LOG.debug("Flink maintenance lock table already exists"); + return true; + } + + LOG.info("Creating Flink maintenance lock table {}", LOCK_TABLE_NAME); + return conn.prepareStatement(CREATE_LOCK_TABLE_SQL).execute(); + }); + + } catch (SQLTimeoutException e) { + throw new UncheckedSQLException( + e, "Cannot initialize JDBC table maintenance lock: Query timed out"); + } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) { + throw new UncheckedSQLException( + e, "Cannot initialize JDBC table maintenance lock: Connection failed"); + } catch (SQLException e) { + throw new UncheckedSQLException(e, "Cannot initialize JDBC table maintenance lock"); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted in call to initialize"); + } + } + + private static class JdbcLock implements TriggerLockFactory.Lock { + private final JdbcClientPool pool; + private final String lockId; + private final Type type; + + private JdbcLock(JdbcClientPool pool, String lockId, Type type) { + this.pool = pool; + this.lockId = lockId; + this.type = type; + } + + @Override + public boolean tryLock() { + if (isHeld()) { + LOG.info("Lock is already held for {}", this); + return false; + } + + String newInstanceId = UUID.randomUUID().toString(); + try { + return pool.run( + conn -> { + try (PreparedStatement sql = conn.prepareStatement(CREATE_LOCK_SQL)) { + sql.setString(1, type.key); + sql.setString(2, lockId); + sql.setString(3, newInstanceId); + int count = sql.executeUpdate(); + LOG.info( + "Created {} lock with instanceId {} with row count {}", + this, + newInstanceId, + count); + return count == 1; + } + }); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted during tryLock"); + } catch (SQLException e) { + // SQL exception happened when creating the lock. Check if the lock creation was + // successful behind the scenes. + if (newInstanceId.equals(instanceId())) { + return true; + } else { + throw new UncheckedSQLException(e, "Failed to create %s lock", this); + } + } + } + + @SuppressWarnings("checkstyle:NestedTryDepth") + @Override + public boolean isHeld() { + try { + return pool.run( + conn -> { + try (PreparedStatement sql = conn.prepareStatement(GET_LOCK_SQL)) { + sql.setString(1, type.key); + sql.setString(2, lockId); + try (ResultSet rs = sql.executeQuery()) { + return rs.next(); + } + } + }); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted during isHeld"); + } catch (SQLException e) { + // SQL exception happened when getting lock information + throw new UncheckedSQLException(e, "Failed to check the state of the lock %s", this); + } + } + + @SuppressWarnings("checkstyle:NestedTryDepth") + @Override + public void unlock() { + try { + // Possible concurrency issue: + // - `unlock` and `tryLock` happens at the same time when there is an existing lock + // + // Steps: + // 1. `unlock` removes the lock in the database, but there is a temporary connection failure + // 2. `lock` founds that there is no lock, so creates a new lock + // 3. `unlock` retires the lock removal and removes the new lock + // + // To prevent the situation above we fetch the current lockId, and remove the lock + // only with the given id. + String instanceId = instanceId(); + + if (instanceId != null) { + pool.run( + conn -> { + try (PreparedStatement sql = conn.prepareStatement(DELETE_LOCK_SQL)) { + sql.setString(1, type.key); + sql.setString(2, lockId); + sql.setString(3, instanceId); + long count = sql.executeUpdate(); + LOG.info( + "Deleted {} lock with instanceId {} with row count {}", + this, + instanceId, + count); + } catch (SQLException e) { + // SQL exception happened when deleting lock information + throw new UncheckedSQLException( + e, "Failed to delete %s lock with instanceId %s", this, instanceId); + } + + return null; + }); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted during unlock"); + } catch (SQLException e) { + // SQL exception happened when getting/updating lock information + throw new UncheckedSQLException(e, "Failed to remove lock %s", this); + } + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this).add("type", type).add("lockId", lockId).toString(); + } + + @SuppressWarnings("checkstyle:NestedTryDepth") + private String instanceId() { + try { + return pool.run( + conn -> { + try (PreparedStatement sql = conn.prepareStatement(GET_LOCK_SQL)) { + sql.setString(1, type.key); + sql.setString(2, lockId); + try (ResultSet rs = sql.executeQuery()) { + if (rs.next()) { + return rs.getString(1); + } else { + return null; + } + } + } catch (SQLException e) { + // SQL exception happened when getting lock information + throw new UncheckedSQLException(e, "Failed to get lock information for %s", type); + } + }); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted during unlock"); + } catch (SQLException e) { + throw new UncheckedSQLException(e, "Failed to get lock information for %s", type); + } + } + } + + private enum Type { + MAINTENANCE("m"), + RECOVERY("r"); + + private final String key; + + Type(String key) { + this.key = key; + } + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java new file mode 100644 index 000000000000..3c3761ef2f4d --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java @@ -0,0 +1,144 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import java.util.List; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.Counter; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Manages locks and collect {@link org.apache.flink.metrics.Metric} for the Maintenance Tasks. + * + *

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

    + *
  • Every {@link TaskResult} is followed by a {@link Watermark} for normal {@link Trigger}s + *
  • For the {@link Trigger#recovery(long)} {@link Watermark} there is no element to process + *
+ * + * When processing the inputs there are 3 possibilities: + * + *
    + *
  • Normal execution - we receive a {@link TaskResult} and then a {@link Watermark} - unlocking + * the lock is handled by the {@link #processElement(StreamRecord)} + *
  • Recovery without ongoing execution (unlocking the recoveryLock) - we receive the {@link + * Trigger#recovery(long)} {@link Watermark} without any {@link TaskResult} - unlocking the + * {@link TriggerLockFactory#createRecoveryLock()} and a possible {@link + * TriggerLockFactory#createLock()} is handled by the {@link #processWatermark(Watermark)} + * (the {@link #lastProcessedTaskStartEpoch} is 0 in this case) + *
  • Recovery with an ongoing execution - we receive a {@link TaskResult} and then a {@link + * Watermark} - unlocking the {@link TriggerLockFactory#createLock()} is handled by the {@link + * #processElement(StreamRecord)}, unlocking the {@link + * TriggerLockFactory#createRecoveryLock()} is handled by the {@link + * #processWatermark(Watermark)} (the {@link #lastProcessedTaskStartEpoch} is the start time + * of the old task) + *
+ */ +@Internal +public class LockRemover extends AbstractStreamOperator + implements OneInputStreamOperator { + private static final Logger LOG = LoggerFactory.getLogger(LockRemover.class); + + private final TriggerLockFactory lockFactory; + private final List maintenanceTaskNames; + + private transient List succeededTaskResultCounters; + private transient List failedTaskResultCounters; + private transient List taskLastRunDurationMs; + private transient TriggerLockFactory.Lock lock; + private transient TriggerLockFactory.Lock recoveryLock; + private transient long lastProcessedTaskStartEpoch = 0L; + + public LockRemover(TriggerLockFactory lockFactory, List maintenanceTaskNames) { + Preconditions.checkNotNull(lockFactory, "Lock factory should no be null"); + Preconditions.checkArgument( + maintenanceTaskNames != null && !maintenanceTaskNames.isEmpty(), + "Invalid maintenance task names: null or empty"); + + this.lockFactory = lockFactory; + this.maintenanceTaskNames = maintenanceTaskNames; + } + + @Override + public void open() throws Exception { + super.open(); + this.succeededTaskResultCounters = + Lists.newArrayListWithExpectedSize(maintenanceTaskNames.size()); + this.failedTaskResultCounters = Lists.newArrayListWithExpectedSize(maintenanceTaskNames.size()); + this.taskLastRunDurationMs = Lists.newArrayListWithExpectedSize(maintenanceTaskNames.size()); + for (String name : maintenanceTaskNames) { + succeededTaskResultCounters.add( + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.SUCCEEDED_TASK_COUNTER)); + failedTaskResultCounters.add( + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.FAILED_TASK_COUNTER)); + AtomicLong duration = new AtomicLong(0); + taskLastRunDurationMs.add(duration); + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .gauge(TableMaintenanceMetrics.LAST_RUN_DURATION_MS, duration::get); + } + + this.lock = lockFactory.createLock(); + this.recoveryLock = lockFactory.createRecoveryLock(); + } + + @Override + public void processElement(StreamRecord streamRecord) { + TaskResult taskResult = streamRecord.getValue(); + LOG.info( + "Processing result {} for task {}", + taskResult, + maintenanceTaskNames.get(taskResult.taskIndex())); + long duration = System.currentTimeMillis() - taskResult.startEpoch(); + lock.unlock(); + this.lastProcessedTaskStartEpoch = taskResult.startEpoch(); + + // Update the metrics + taskLastRunDurationMs.get(taskResult.taskIndex()).set(duration); + if (taskResult.success()) { + succeededTaskResultCounters.get(taskResult.taskIndex()).inc(); + } else { + failedTaskResultCounters.get(taskResult.taskIndex()).inc(); + } + } + + @Override + public void processWatermark(Watermark mark) { + if (mark.getTimestamp() > lastProcessedTaskStartEpoch) { + lock.unlock(); + recoveryLock.unlock(); + } + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java index d74b2349b1de..89efffa15f16 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java @@ -43,7 +43,7 @@ /** Monitors an Iceberg table for changes */ @Internal -public class MonitorSource extends SingleThreadedIteratorSource { +class MonitorSource extends SingleThreadedIteratorSource { private static final Logger LOG = LoggerFactory.getLogger(MonitorSource.class); private final TableLoader tableLoader; @@ -58,7 +58,7 @@ public class MonitorSource extends SingleThreadedIteratorSource { * @param rateLimiterStrategy limits the frequency the table is checked * @param maxReadBack sets the number of snapshots read before stopping change collection */ - public MonitorSource( + MonitorSource( TableLoader tableLoader, RateLimiterStrategy rateLimiterStrategy, long maxReadBack) { Preconditions.checkNotNull(tableLoader, "Table loader should no be null"); Preconditions.checkNotNull(rateLimiterStrategy, "Rate limiter strategy should no be null"); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java index 452ed80ed0e5..773b34b6c495 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java @@ -29,19 +29,29 @@ /** Event describing changes in an Iceberg table */ @Internal class TableChange { - private int dataFileNum; - private int deleteFileNum; - private long dataFileSize; - private long deleteFileSize; - private int commitNum; + private int dataFileCount; + private long dataFileSizeInBytes; + private int posDeleteFileCount; + private long posDeleteRecordCount; + private int eqDeleteFileCount; + private long eqDeleteRecordCount; + private int commitCount; TableChange( - int dataFileNum, int deleteFileNum, long dataFileSize, long deleteFileSize, int commitNum) { - this.dataFileNum = dataFileNum; - this.deleteFileNum = deleteFileNum; - this.dataFileSize = dataFileSize; - this.deleteFileSize = deleteFileSize; - this.commitNum = commitNum; + int dataFileCount, + long dataFileSizeInBytes, + int posDeleteFileCount, + long posDeleteRecordCount, + int eqDeleteFileCount, + long eqDeleteRecordCount, + int commitCount) { + this.dataFileCount = dataFileCount; + this.dataFileSizeInBytes = dataFileSizeInBytes; + this.posDeleteFileCount = posDeleteFileCount; + this.posDeleteRecordCount = posDeleteRecordCount; + this.eqDeleteFileCount = eqDeleteFileCount; + this.eqDeleteRecordCount = eqDeleteRecordCount; + this.commitCount = commitCount; } TableChange(Snapshot snapshot, FileIO io) { @@ -50,63 +60,96 @@ class TableChange { dataFiles.forEach( dataFile -> { - this.dataFileNum++; - this.dataFileSize += dataFile.fileSizeInBytes(); + this.dataFileCount++; + this.dataFileSizeInBytes += dataFile.fileSizeInBytes(); }); deleteFiles.forEach( deleteFile -> { - this.deleteFileNum++; - this.deleteFileSize += deleteFile.fileSizeInBytes(); + switch (deleteFile.content()) { + case POSITION_DELETES: + this.posDeleteFileCount++; + this.posDeleteRecordCount += deleteFile.recordCount(); + break; + case EQUALITY_DELETES: + this.eqDeleteFileCount++; + this.eqDeleteRecordCount += deleteFile.recordCount(); + break; + default: + throw new IllegalArgumentException("Unexpected delete file content: " + deleteFile); + } }); - this.commitNum = 1; + this.commitCount = 1; } static TableChange empty() { - return new TableChange(0, 0, 0L, 0L, 0); + return new TableChange(0, 0L, 0, 0L, 0, 0L, 0); } - int dataFileNum() { - return dataFileNum; + static Builder builder() { + return new Builder(); } - int deleteFileNum() { - return deleteFileNum; + int dataFileCount() { + return dataFileCount; } - long dataFileSize() { - return dataFileSize; + long dataFileSizeInBytes() { + return dataFileSizeInBytes; } - long deleteFileSize() { - return deleteFileSize; + int posDeleteFileCount() { + return posDeleteFileCount; } - public int commitNum() { - return commitNum; + long posDeleteRecordCount() { + return posDeleteRecordCount; + } + + int eqDeleteFileCount() { + return eqDeleteFileCount; + } + + long eqDeleteRecordCount() { + return eqDeleteRecordCount; + } + + public int commitCount() { + return commitCount; } public void merge(TableChange other) { - this.dataFileNum += other.dataFileNum; - this.deleteFileNum += other.deleteFileNum; - this.dataFileSize += other.dataFileSize; - this.deleteFileSize += other.deleteFileSize; - this.commitNum += other.commitNum; + this.dataFileCount += other.dataFileCount; + this.dataFileSizeInBytes += other.dataFileSizeInBytes; + this.posDeleteFileCount += other.posDeleteFileCount; + this.posDeleteRecordCount += other.posDeleteRecordCount; + this.eqDeleteFileCount += other.eqDeleteFileCount; + this.eqDeleteRecordCount += other.eqDeleteRecordCount; + this.commitCount += other.commitCount; } TableChange copy() { - return new TableChange(dataFileNum, deleteFileNum, dataFileSize, deleteFileSize, commitNum); + return new TableChange( + dataFileCount, + dataFileSizeInBytes, + posDeleteFileCount, + posDeleteRecordCount, + eqDeleteFileCount, + eqDeleteRecordCount, + commitCount); } @Override public String toString() { return MoreObjects.toStringHelper(this) - .add("dataFileNum", dataFileNum) - .add("deleteFileNum", deleteFileNum) - .add("dataFileSize", dataFileSize) - .add("deleteFileSize", deleteFileSize) - .add("commitNum", commitNum) + .add("dataFileCount", dataFileCount) + .add("dataFileSizeInBytes", dataFileSizeInBytes) + .add("posDeleteFileCount", posDeleteFileCount) + .add("posDeleteRecordCount", posDeleteRecordCount) + .add("eqDeleteFileCount", eqDeleteFileCount) + .add("eqDeleteRecordCount", eqDeleteRecordCount) + .add("commitCount", commitCount) .toString(); } @@ -119,15 +162,82 @@ public boolean equals(Object other) { } TableChange that = (TableChange) other; - return dataFileNum == that.dataFileNum - && deleteFileNum == that.deleteFileNum - && dataFileSize == that.dataFileSize - && deleteFileSize == that.deleteFileSize - && commitNum == that.commitNum; + return dataFileCount == that.dataFileCount + && dataFileSizeInBytes == that.dataFileSizeInBytes + && posDeleteFileCount == that.posDeleteFileCount + && posDeleteRecordCount == that.posDeleteRecordCount + && eqDeleteFileCount == that.eqDeleteFileCount + && eqDeleteRecordCount == that.eqDeleteRecordCount + && commitCount == that.commitCount; } @Override public int hashCode() { - return Objects.hash(dataFileNum, deleteFileNum, dataFileSize, deleteFileSize, commitNum); + return Objects.hash( + dataFileCount, + dataFileSizeInBytes, + posDeleteFileCount, + posDeleteRecordCount, + eqDeleteFileCount, + eqDeleteRecordCount, + commitCount); + } + + static class Builder { + private int dataFileCount = 0; + private long dataFileSizeInBytes = 0L; + private int posDeleteFileCount = 0; + private long posDeleteRecordCount = 0L; + private int eqDeleteFileCount = 0; + private long eqDeleteRecordCount = 0L; + private int commitCount = 0; + + private Builder() {} + + public Builder dataFileCount(int newDataFileCount) { + this.dataFileCount = newDataFileCount; + return this; + } + + public Builder dataFileSizeInBytes(long newDataFileSizeInBytes) { + this.dataFileSizeInBytes = newDataFileSizeInBytes; + return this; + } + + public Builder posDeleteFileCount(int newPosDeleteFileCount) { + this.posDeleteFileCount = newPosDeleteFileCount; + return this; + } + + public Builder posDeleteRecordCount(long newPosDeleteRecordCount) { + this.posDeleteRecordCount = newPosDeleteRecordCount; + return this; + } + + public Builder eqDeleteFileCount(int newEqDeleteFileCount) { + this.eqDeleteFileCount = newEqDeleteFileCount; + return this; + } + + public Builder eqDeleteRecordCount(long newEqDeleteRecordCount) { + this.eqDeleteRecordCount = newEqDeleteRecordCount; + return this; + } + + public Builder commitCount(int newCommitCount) { + this.commitCount = newCommitCount; + return this; + } + + public TableChange build() { + return new TableChange( + dataFileCount, + dataFileSizeInBytes, + posDeleteFileCount, + posDeleteRecordCount, + eqDeleteFileCount, + eqDeleteRecordCount, + commitCount); + } } } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java new file mode 100644 index 000000000000..1a04461aed43 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +public class TableMaintenanceMetrics { + public static final String GROUP_KEY = "maintenanceTask"; + public static final String GROUP_VALUE_DEFAULT = "maintenanceTask"; + + // TriggerManager metrics + public static final String RATE_LIMITER_TRIGGERED = "rateLimiterTriggered"; + public static final String CONCURRENT_RUN_THROTTLED = "concurrentRunThrottled"; + public static final String TRIGGERED = "triggered"; + public static final String NOTHING_TO_TRIGGER = "nothingToTrigger"; + + // LockRemover metrics + public static final String SUCCEEDED_TASK_COUNTER = "succeededTasks"; + public static final String FAILED_TASK_COUNTER = "failedTasks"; + public static final String LAST_RUN_DURATION_MS = "lastRunDurationMs"; + + private TableMaintenanceMetrics() { + // do not instantiate + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TaskResult.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TaskResult.java new file mode 100644 index 000000000000..06f10f1c1d68 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TaskResult.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import java.util.List; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +/** The result of a single Maintenance Task. */ +@Internal +public class TaskResult { + private final int taskIndex; + private final long startEpoch; + private final boolean success; + private final List exceptions; + + public TaskResult(int taskIndex, long startEpoch, boolean success, List exceptions) { + this.taskIndex = taskIndex; + this.startEpoch = startEpoch; + this.success = success; + this.exceptions = exceptions; + } + + public int taskIndex() { + return taskIndex; + } + + public long startEpoch() { + return startEpoch; + } + + public boolean success() { + return success; + } + + public List exceptions() { + return exceptions; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("taskIndex", taskIndex) + .add("startEpoch", startEpoch) + .add("success", success) + .add("exceptions", exceptions) + .toString(); + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/Trigger.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/Trigger.java new file mode 100644 index 000000000000..85c6c8dbdd55 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/Trigger.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +@Internal +class Trigger { + private final long timestamp; + private final SerializableTable table; + private final Integer taskId; + private final boolean isRecovery; + + private Trigger(long timestamp, SerializableTable table, Integer taskId, boolean isRecovery) { + this.timestamp = timestamp; + this.table = table; + this.taskId = taskId; + this.isRecovery = isRecovery; + } + + static Trigger create(long timestamp, SerializableTable table, int taskId) { + return new Trigger(timestamp, table, taskId, false); + } + + static Trigger recovery(long timestamp) { + return new Trigger(timestamp, null, null, true); + } + + long timestamp() { + return timestamp; + } + + SerializableTable table() { + return table; + } + + Integer taskId() { + return taskId; + } + + boolean isRecovery() { + return isRecovery; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("timestamp", timestamp) + .add("table", table == null ? null : table.name()) + .add("taskId", taskId) + .add("isRecovery", isRecovery) + .toString(); + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java new file mode 100644 index 000000000000..dba33b22a42a --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import java.io.Serializable; +import java.time.Duration; +import java.util.List; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Internal +class TriggerEvaluator implements Serializable { + private static final Logger LOG = LoggerFactory.getLogger(TriggerEvaluator.class); + private final List predicates; + + private TriggerEvaluator(List predicates) { + Preconditions.checkArgument(!predicates.isEmpty(), "Provide at least 1 condition."); + + this.predicates = predicates; + } + + boolean check(TableChange event, long lastTimeMs, long currentTimeMs) { + boolean result = + predicates.stream().anyMatch(p -> p.evaluate(event, lastTimeMs, currentTimeMs)); + LOG.debug( + "Checking event: {}, at {}, last: {} with result: {}", + event, + currentTimeMs, + lastTimeMs, + result); + return result; + } + + static class Builder implements Serializable { + private Integer dataFileCount; + private Long dataFileSizeInBytes; + private Integer posDeleteFileCount; + private Long posDeleteRecordCount; + private Integer eqDeleteFileCount; + private Long eqDeleteRecordCount; + private Integer commitCount; + private Duration timeout; + + public Builder dataFileCount(int newDataFileCount) { + this.dataFileCount = newDataFileCount; + return this; + } + + public Builder dataFileSizeInBytes(long neDataFileSizeInBytes) { + this.dataFileSizeInBytes = neDataFileSizeInBytes; + return this; + } + + public Builder posDeleteFileCount(int newPosDeleteFileCount) { + this.posDeleteFileCount = newPosDeleteFileCount; + return this; + } + + public Builder posDeleteRecordCount(long newPosDeleteRecordCount) { + this.posDeleteRecordCount = newPosDeleteRecordCount; + return this; + } + + public Builder eqDeleteFileCount(int newEqDeleteFileCount) { + this.eqDeleteFileCount = newEqDeleteFileCount; + return this; + } + + public Builder eqDeleteRecordCount(long newEqDeleteRecordCount) { + this.eqDeleteRecordCount = newEqDeleteRecordCount; + return this; + } + + public Builder commitCount(int newCommitCount) { + this.commitCount = newCommitCount; + return this; + } + + Builder timeout(Duration newTimeout) { + this.timeout = newTimeout; + return this; + } + + TriggerEvaluator build() { + List predicates = Lists.newArrayList(); + if (dataFileCount != null) { + predicates.add((change, unused, unused2) -> change.dataFileCount() >= dataFileCount); + } + + if (dataFileSizeInBytes != null) { + predicates.add( + (change, unused, unused2) -> change.dataFileSizeInBytes() >= dataFileSizeInBytes); + } + + if (posDeleteFileCount != null) { + predicates.add( + (change, unused, unused2) -> change.posDeleteFileCount() >= posDeleteFileCount); + } + + if (posDeleteRecordCount != null) { + predicates.add( + (change, unused, unused2) -> change.posDeleteRecordCount() >= posDeleteRecordCount); + } + + if (eqDeleteFileCount != null) { + predicates.add( + (change, unused, unused2) -> change.eqDeleteFileCount() >= eqDeleteFileCount); + } + + if (eqDeleteRecordCount != null) { + predicates.add( + (change, unused, unused2) -> change.eqDeleteRecordCount() >= eqDeleteRecordCount); + } + + if (commitCount != null) { + predicates.add((change, unused, unused2) -> change.commitCount() >= commitCount); + } + + if (timeout != null) { + predicates.add( + (change, lastTimeMs, currentTimeMs) -> + currentTimeMs - lastTimeMs >= timeout.toMillis()); + } + + return new TriggerEvaluator(predicates); + } + } + + private interface Predicate extends Serializable { + boolean evaluate(TableChange event, long lastTimeMs, long currentTimeMs); + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerLockFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerLockFactory.java new file mode 100644 index 000000000000..329223d27ccf --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerLockFactory.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import java.io.Closeable; +import java.io.Serializable; +import org.apache.flink.annotation.Experimental; + +/** Lock interface for handling locks for the Flink Table Maintenance jobs. */ +@Experimental +public interface TriggerLockFactory extends Serializable, Closeable { + void open(); + + Lock createLock(); + + Lock createRecoveryLock(); + + interface Lock { + /** + * Tries to acquire a lock with a given key. Anyone already holding a lock would prevent + * acquiring this lock. Not reentrant. + * + *

Called by {@link TriggerManager}. Implementations could assume that are no concurrent + * calls for this method. + * + * @return true if the lock is acquired by this job, false if the lock + * is already held by someone + */ + boolean tryLock(); + + /** + * Checks if the lock is already taken. + * + * @return true if the lock is held by someone + */ + boolean isHeld(); + + // TODO: Fix the link to the LockRemover when we have a final name and implementation + /** + * Releases the lock. Should not fail if the lock is not held by anyone. + * + *

Called by LockRemover. Implementations could assume that are no concurrent calls for this + * method. + */ + void unlock(); + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java new file mode 100644 index 000000000000..dc95b27af0a6 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java @@ -0,0 +1,341 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.streaming.api.TimerService; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.functions.KeyedProcessFunction; +import org.apache.flink.util.Collector; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * TriggerManager starts the Maintenance Tasks by emitting {@link Trigger} messages which are + * calculated based on the incoming {@link TableChange} messages. The TriggerManager keeps track of + * the changes since the last run of the Maintenance Tasks and triggers a new run based on the + * result of the {@link TriggerEvaluator}. + * + *

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

The TriggerManager should run as a global operator. {@link KeyedProcessFunction} is used, so + * the timer functions are available, but the key is not used. + */ +@Internal +class TriggerManager extends KeyedProcessFunction + implements CheckpointedFunction { + private static final Logger LOG = LoggerFactory.getLogger(TriggerManager.class); + + private final TableLoader tableLoader; + private final TriggerLockFactory lockFactory; + private final List maintenanceTaskNames; + private final List evaluators; + private final long minFireDelayMs; + private final long lockCheckDelayMs; + private transient Counter rateLimiterTriggeredCounter; + private transient Counter concurrentRunThrottledCounter; + private transient Counter nothingToTriggerCounter; + private transient List triggerCounters; + private transient ValueState nextEvaluationTimeState; + private transient ListState accumulatedChangesState; + private transient ListState lastTriggerTimesState; + private transient Long nextEvaluationTime; + private transient List accumulatedChanges; + private transient List lastTriggerTimes; + private transient TriggerLockFactory.Lock lock; + private transient TriggerLockFactory.Lock recoveryLock; + private transient boolean shouldRestoreTasks = false; + private transient boolean inited = false; + // To keep the task scheduling fair we keep the last triggered task position in memory. + // If we find a task to trigger, then we run it, but after it is finished, we start from the given + // position to prevent "starvation" of the tasks. + // When there is nothing to trigger, we start from the beginning, as the order of the tasks might + // be important (RewriteDataFiles first, and then RewriteManifestFiles later) + private transient int startsFrom = 0; + private transient boolean triggered = false; + + TriggerManager( + TableLoader tableLoader, + TriggerLockFactory lockFactory, + List maintenanceTaskNames, + List evaluators, + long minFireDelayMs, + long lockCheckDelayMs) { + Preconditions.checkNotNull(tableLoader, "Table loader should no be null"); + Preconditions.checkNotNull(lockFactory, "Lock factory should no be null"); + Preconditions.checkArgument( + maintenanceTaskNames != null && !maintenanceTaskNames.isEmpty(), + "Invalid maintenance task names: null or empty"); + Preconditions.checkArgument( + evaluators != null && !evaluators.isEmpty(), "Invalid evaluators: null or empty"); + Preconditions.checkArgument( + maintenanceTaskNames.size() == evaluators.size(), + "Provide a name and evaluator for all of the maintenance tasks"); + Preconditions.checkArgument(minFireDelayMs > 0, "Minimum fire delay should be at least 1."); + Preconditions.checkArgument( + lockCheckDelayMs > 0, "Minimum lock delay rate should be at least 1 ms."); + + this.tableLoader = tableLoader; + this.lockFactory = lockFactory; + this.maintenanceTaskNames = maintenanceTaskNames; + this.evaluators = evaluators; + this.minFireDelayMs = minFireDelayMs; + this.lockCheckDelayMs = lockCheckDelayMs; + } + + @Override + public void open(Configuration parameters) throws Exception { + this.rateLimiterTriggeredCounter = + getRuntimeContext() + .getMetricGroup() + .addGroup( + TableMaintenanceMetrics.GROUP_KEY, TableMaintenanceMetrics.GROUP_VALUE_DEFAULT) + .counter(TableMaintenanceMetrics.RATE_LIMITER_TRIGGERED); + this.concurrentRunThrottledCounter = + getRuntimeContext() + .getMetricGroup() + .addGroup( + TableMaintenanceMetrics.GROUP_KEY, TableMaintenanceMetrics.GROUP_VALUE_DEFAULT) + .counter(TableMaintenanceMetrics.CONCURRENT_RUN_THROTTLED); + this.nothingToTriggerCounter = + getRuntimeContext() + .getMetricGroup() + .addGroup( + TableMaintenanceMetrics.GROUP_KEY, TableMaintenanceMetrics.GROUP_VALUE_DEFAULT) + .counter(TableMaintenanceMetrics.NOTHING_TO_TRIGGER); + this.triggerCounters = + maintenanceTaskNames.stream() + .map( + name -> + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.TRIGGERED)) + .collect(Collectors.toList()); + + this.nextEvaluationTimeState = + getRuntimeContext() + .getState(new ValueStateDescriptor<>("triggerManagerNextTriggerTime", Types.LONG)); + this.accumulatedChangesState = + getRuntimeContext() + .getListState( + new ListStateDescriptor<>( + "triggerManagerAccumulatedChange", TypeInformation.of(TableChange.class))); + this.lastTriggerTimesState = + getRuntimeContext() + .getListState(new ListStateDescriptor<>("triggerManagerLastTriggerTime", Types.LONG)); + + tableLoader.open(); + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + if (inited) { + // Only store state if initialized + nextEvaluationTimeState.update(nextEvaluationTime); + accumulatedChangesState.update(accumulatedChanges); + lastTriggerTimesState.update(lastTriggerTimes); + LOG.info( + "Storing state: nextEvaluationTime {}, accumulatedChanges {}, lastTriggerTimes {}", + nextEvaluationTime, + accumulatedChanges, + lastTriggerTimes); + } else { + LOG.info("Not initialized, state is not stored"); + } + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + LOG.info("Initializing state restored: {}", context.isRestored()); + lockFactory.open(); + this.lock = lockFactory.createLock(); + this.recoveryLock = lockFactory.createRecoveryLock(); + if (context.isRestored()) { + shouldRestoreTasks = true; + } + } + + @Override + public void processElement(TableChange change, Context ctx, Collector out) + throws Exception { + init(out, ctx.timerService()); + + accumulatedChanges.forEach(tableChange -> tableChange.merge(change)); + + long current = ctx.timerService().currentProcessingTime(); + if (nextEvaluationTime == null) { + checkAndFire(current, ctx.timerService(), out); + } else { + LOG.info( + "Trigger manager rate limiter triggered current: {}, next: {}, accumulated changes: {}", + current, + nextEvaluationTime, + accumulatedChanges); + rateLimiterTriggeredCounter.inc(); + } + } + + @Override + public void onTimer(long timestamp, OnTimerContext ctx, Collector out) throws Exception { + init(out, ctx.timerService()); + this.nextEvaluationTime = null; + checkAndFire(ctx.timerService().currentProcessingTime(), ctx.timerService(), out); + } + + @Override + public void close() throws IOException { + tableLoader.close(); + lockFactory.close(); + } + + private void checkAndFire(long current, TimerService timerService, Collector out) { + if (shouldRestoreTasks) { + if (recoveryLock.isHeld()) { + // Recovered tasks in progress. Skip trigger check + LOG.debug("The recovery lock is still held at {}", current); + schedule(timerService, current + lockCheckDelayMs); + return; + } else { + LOG.info("The recovery is finished at {}", current); + shouldRestoreTasks = false; + } + } + + Integer taskToStart = + nextTrigger(evaluators, accumulatedChanges, lastTriggerTimes, current, startsFrom); + if (taskToStart == null) { + // Nothing to execute + if (!triggered) { + nothingToTriggerCounter.inc(); + LOG.debug("Nothing to execute at {} for collected: {}", current, accumulatedChanges); + } else { + LOG.debug("Execution check finished"); + } + + // Next time start from the beginning + startsFrom = 0; + triggered = false; + return; + } + + if (lock.tryLock()) { + TableChange change = accumulatedChanges.get(taskToStart); + SerializableTable table = + (SerializableTable) SerializableTable.copyOf(tableLoader.loadTable()); + out.collect(Trigger.create(current, table, taskToStart)); + LOG.debug("Fired event with time: {}, collected: {} for {}", current, change, table.name()); + triggerCounters.get(taskToStart).inc(); + accumulatedChanges.set(taskToStart, TableChange.empty()); + lastTriggerTimes.set(taskToStart, current); + schedule(timerService, current + minFireDelayMs); + startsFrom = (taskToStart + 1) % evaluators.size(); + triggered = true; + } else { + // A task is already running, waiting for it to finish + LOG.info("Failed to acquire lock. Delaying task to {}", current + lockCheckDelayMs); + + startsFrom = taskToStart; + concurrentRunThrottledCounter.inc(); + schedule(timerService, current + lockCheckDelayMs); + } + + timerService.registerProcessingTimeTimer(nextEvaluationTime); + } + + private void schedule(TimerService timerService, long time) { + this.nextEvaluationTime = time; + timerService.registerProcessingTimeTimer(time); + } + + private static Integer nextTrigger( + List evaluators, + List changes, + List lastTriggerTimes, + long currentTime, + int startPos) { + int current = startPos; + do { + if (evaluators + .get(current) + .check(changes.get(current), lastTriggerTimes.get(current), currentTime)) { + return current; + } + + current = (current + 1) % evaluators.size(); + } while (current != startPos); + + return null; + } + + private void init(Collector out, TimerService timerService) throws Exception { + if (!inited) { + long current = timerService.currentProcessingTime(); + + // Initialize from state + this.nextEvaluationTime = nextEvaluationTimeState.value(); + this.accumulatedChanges = Lists.newArrayList(accumulatedChangesState.get()); + this.lastTriggerTimes = Lists.newArrayList(lastTriggerTimesState.get()); + + // Initialize if the state was empty + if (accumulatedChanges.isEmpty()) { + for (int i = 0; i < evaluators.size(); ++i) { + accumulatedChanges.add(TableChange.empty()); + lastTriggerTimes.add(current); + } + } + + if (shouldRestoreTasks) { + // When the job state is restored, there could be ongoing tasks. + // To prevent collision with the new triggers the following is done: + // - add a recovery lock + // - fire a recovery trigger + // This ensures that the tasks of the previous trigger are executed, and the lock is removed + // in the end. The result of the 'tryLock' is ignored as an already existing lock prevents + // collisions as well. + recoveryLock.tryLock(); + out.collect(Trigger.recovery(current)); + if (nextEvaluationTime == null) { + schedule(timerService, current + minFireDelayMs); + } + } + + inited = true; + } + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java index c7e8a2dea7cb..9571efdc5268 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java @@ -33,9 +33,14 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; class FlinkManifestUtil { + + private static final Logger LOG = LoggerFactory.getLogger(FlinkManifestUtil.class); private static final int FORMAT_V2 = 2; private static final Long DUMMY_SNAPSHOT_ID = 0L; @@ -129,4 +134,26 @@ static WriteResult readCompletedFiles( return builder.addReferencedDataFiles(deltaManifests.referencedDataFiles()).build(); } + + static void deleteCommittedManifests( + Table table, List manifests, String newFlinkJobId, long checkpointId) { + for (ManifestFile manifest : manifests) { + try { + table.io().deleteFile(manifest.path()); + } catch (Exception e) { + // The flink manifests cleaning failure shouldn't abort the completed checkpoint. + String details = + MoreObjects.toStringHelper(FlinkManifestUtil.class) + .add("tableName", table.name()) + .add("flinkJobId", newFlinkJobId) + .add("checkpointId", checkpointId) + .add("manifestPath", manifest.path()) + .toString(); + LOG.warn( + "The iceberg transaction has been committed, but we failed to clean the temporary flink manifests: {}", + details, + e); + } + } + } } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java index 769af7d77140..e862e88c968c 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java @@ -53,15 +53,20 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Partitioning; import org.apache.iceberg.Schema; import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.SortOrder; import org.apache.iceberg.Table; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.FlinkWriteConf; import org.apache.iceberg.flink.FlinkWriteOptions; import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.sink.shuffle.DataStatisticsOperatorFactory; +import org.apache.iceberg.flink.sink.shuffle.RangePartitioner; +import org.apache.iceberg.flink.sink.shuffle.StatisticsOrRecord; +import org.apache.iceberg.flink.sink.shuffle.StatisticsType; import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; -import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -129,7 +134,7 @@ public static Builder forRowData(DataStream input) { return new Builder().forRowData(input); } - public static class Builder { + public static class Builder implements IcebergSinkBuilder { private Function> inputCreator = null; private TableLoader tableLoader; private Table table; @@ -174,6 +179,7 @@ private Builder forMapperOutputType( * @param newTable the loaded iceberg table instance. * @return {@link Builder} to connect the iceberg table. */ + @Override public Builder table(Table newTable) { this.table = newTable; return this; @@ -187,6 +193,7 @@ public Builder table(Table newTable) { * @param newTableLoader to load iceberg table inside tasks. * @return {@link Builder} to connect the iceberg table. */ + @Override public Builder tableLoader(TableLoader newTableLoader) { this.tableLoader = newTableLoader; return this; @@ -205,21 +212,25 @@ public Builder set(String property, String value) { * Set the write properties for Flink sink. View the supported properties in {@link * FlinkWriteOptions} */ + @Override public Builder setAll(Map properties) { writeOptions.putAll(properties); return this; } + @Override public Builder tableSchema(TableSchema newTableSchema) { this.tableSchema = newTableSchema; return this; } + @Override public Builder overwrite(boolean newOverwrite) { writeOptions.put(FlinkWriteOptions.OVERWRITE_MODE.key(), Boolean.toString(newOverwrite)); return this; } + @Override public Builder flinkConf(ReadableConfig config) { this.readableConfig = config; return this; @@ -232,22 +243,77 @@ public Builder flinkConf(ReadableConfig config) { * @param mode to specify the write distribution mode. * @return {@link Builder} to connect the iceberg table. */ + @Override public Builder distributionMode(DistributionMode mode) { - Preconditions.checkArgument( - !DistributionMode.RANGE.equals(mode), - "Flink does not support 'range' write distribution mode now."); if (mode != null) { writeOptions.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), mode.modeName()); } return this; } + /** + * Range distribution needs to collect statistics about data distribution to properly shuffle + * the records in relatively balanced way. In general, low cardinality should use {@link + * StatisticsType#Map} and high cardinality should use {@link StatisticsType#Sketch} Refer to + * {@link StatisticsType} Javadoc for more details. + * + *

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

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

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

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

Default is {@code 0.0%}. + */ + public Builder rangeDistributionSortKeyBaseWeight(double weight) { + writeOptions.put( + FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT.key(), Double.toString(weight)); + return this; + } + /** * Configuring the write parallel number for iceberg stream writer. * * @param newWriteParallelism the number of parallel iceberg stream writer. * @return {@link Builder} to connect the iceberg table. */ + @Override public Builder writeParallelism(int newWriteParallelism) { writeOptions.put( FlinkWriteOptions.WRITE_PARALLELISM.key(), Integer.toString(newWriteParallelism)); @@ -263,6 +329,7 @@ public Builder writeParallelism(int newWriteParallelism) { * @param enabled indicate whether it should transform all INSERT/UPDATE_AFTER events to UPSERT. * @return {@link Builder} to connect the iceberg table. */ + @Override public Builder upsert(boolean enabled) { writeOptions.put(FlinkWriteOptions.WRITE_UPSERT_ENABLED.key(), Boolean.toString(enabled)); return this; @@ -274,6 +341,7 @@ public Builder upsert(boolean enabled) { * @param columns defines the iceberg table's key. * @return {@link Builder} to connect the iceberg table. */ + @Override public Builder equalityFieldColumns(List columns) { this.equalityFieldColumns = columns; return this; @@ -318,6 +386,7 @@ public Builder setSnapshotProperty(String property, String value) { return this; } + @Override public Builder toBranch(String branch) { writeOptions.put(FlinkWriteOptions.BRANCH.key(), branch); return this; @@ -347,20 +416,23 @@ private DataStreamSink chainIcebergOperators() { flinkWriteConf = new FlinkWriteConf(table, writeOptions, readableConfig); // Find out the equality field id list based on the user-provided equality field column names. - List equalityFieldIds = checkAndGetEqualityFieldIds(); + List equalityFieldIds = + SinkUtil.checkAndGetEqualityFieldIds(table, equalityFieldColumns); - // Convert the requested flink table schema to flink row type. RowType flinkRowType = toFlinkRowType(table.schema(), tableSchema); + int writerParallelism = + flinkWriteConf.writeParallelism() == null + ? rowDataInput.getParallelism() + : flinkWriteConf.writeParallelism(); // Distribute the records from input data stream based on the write.distribution-mode and // equality fields. DataStream distributeStream = - distributeDataStream( - rowDataInput, equalityFieldIds, table.spec(), table.schema(), flinkRowType); + distributeDataStream(rowDataInput, equalityFieldIds, flinkRowType, writerParallelism); // Add parallel writers that append rows to files - SingleOutputStreamOperator writerStream = - appendWriter(distributeStream, flinkRowType, equalityFieldIds); + SingleOutputStreamOperator writerStream = + appendWriter(distributeStream, flinkRowType, equalityFieldIds, writerParallelism); // Add single-parallelism committer that commits files // after successful checkpoint or end of input @@ -375,6 +447,7 @@ private DataStreamSink chainIcebergOperators() { * * @return {@link DataStreamSink} for sink. */ + @Override public DataStreamSink append() { return chainIcebergOperators(); } @@ -426,7 +499,7 @@ private DataStreamSink appendDummySink( } private SingleOutputStreamOperator appendCommitter( - SingleOutputStreamOperator writerStream) { + SingleOutputStreamOperator writerStream) { IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter( tableLoader, @@ -446,8 +519,11 @@ private SingleOutputStreamOperator appendCommitter( return committerStream; } - private SingleOutputStreamOperator appendWriter( - DataStream input, RowType flinkRowType, List equalityFieldIds) { + private SingleOutputStreamOperator appendWriter( + DataStream input, + RowType flinkRowType, + List equalityFieldIds, + int writerParallelism) { // Validate the equality fields and partition fields if we enable the upsert mode. if (flinkWriteConf.upsertMode()) { Preconditions.checkState( @@ -481,17 +557,13 @@ private SingleOutputStreamOperator appendWriter( IcebergStreamWriter streamWriter = createStreamWriter(tableSupplier, flinkWriteConf, flinkRowType, equalityFieldIds); - int parallelism = - flinkWriteConf.writeParallelism() == null - ? input.getParallelism() - : flinkWriteConf.writeParallelism(); - SingleOutputStreamOperator writerStream = + SingleOutputStreamOperator writerStream = input .transform( operatorName(ICEBERG_STREAM_WRITER_NAME), - TypeInformation.of(WriteResult.class), + TypeInformation.of(FlinkWriteResult.class), streamWriter) - .setParallelism(parallelism); + .setParallelism(writerParallelism); if (uidPrefix != null) { writerStream = writerStream.uid(uidPrefix + "-writer"); } @@ -501,12 +573,15 @@ private SingleOutputStreamOperator appendWriter( private DataStream distributeDataStream( DataStream input, List equalityFieldIds, - PartitionSpec partitionSpec, - Schema iSchema, - RowType flinkRowType) { + RowType flinkRowType, + int writerParallelism) { DistributionMode writeMode = flinkWriteConf.distributionMode(); - LOG.info("Write distribution mode is '{}'", writeMode.modeName()); + + Schema iSchema = table.schema(); + PartitionSpec partitionSpec = table.spec(); + SortOrder sortOrder = table.sortOrder(); + switch (writeMode) { case NONE: if (equalityFieldIds.isEmpty()) { @@ -548,21 +623,52 @@ private DataStream distributeDataStream( } case RANGE: - if (equalityFieldIds.isEmpty()) { + // Ideally, exception should be thrown in the combination of range distribution and + // equality fields. Primary key case should use hash distribution mode. + // Keep the current behavior of falling back to keyBy for backward compatibility. + if (!equalityFieldIds.isEmpty()) { LOG.warn( - "Fallback to use 'none' distribution mode, because there are no equality fields set " - + "and {}=range is not supported yet in flink", - WRITE_DISTRIBUTION_MODE); - return input; - } else { - LOG.info( - "Distribute rows by equality fields, because there are equality fields set " - + "and{}=range is not supported yet in flink", + "Hash distribute rows by equality fields, even though {}=range is set. " + + "Range distribution for primary keys are not always safe in " + + "Flink streaming writer.", WRITE_DISTRIBUTION_MODE); return input.keyBy( new EqualityFieldKeySelector(iSchema, flinkRowType, equalityFieldIds)); } + // range distribute by partition key or sort key if table has an SortOrder + Preconditions.checkState( + sortOrder.isSorted() || partitionSpec.isPartitioned(), + "Invalid write distribution mode: range. Need to define sort order or partition spec."); + if (sortOrder.isUnsorted()) { + sortOrder = Partitioning.sortOrderFor(partitionSpec); + LOG.info("Construct sort order from partition spec"); + } + + LOG.info("Range distribute rows by sort order: {}", sortOrder); + StatisticsType statisticsType = flinkWriteConf.rangeDistributionStatisticsType(); + SingleOutputStreamOperator shuffleStream = + input + .transform( + operatorName("range-shuffle"), + TypeInformation.of(StatisticsOrRecord.class), + new DataStatisticsOperatorFactory( + iSchema, + sortOrder, + writerParallelism, + statisticsType, + flinkWriteConf.rangeDistributionSortKeyBaseWeight())) + // Set the parallelism same as input operator to encourage chaining + .setParallelism(input.getParallelism()); + if (uidPrefix != null) { + shuffleStream = shuffleStream.uid(uidPrefix + "-shuffle"); + } + + return shuffleStream + .partitionCustom(new RangePartitioner(iSchema, sortOrder), r -> r) + .filter(StatisticsOrRecord::hasRecord) + .map(StatisticsOrRecord::record); + default: throw new RuntimeException("Unrecognized " + WRITE_DISTRIBUTION_MODE + ": " + writeMode); } @@ -577,12 +683,9 @@ static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) { TypeUtil.validateWriteSchema(schema, writeSchema, true, true); // We use this flink schema to read values from RowData. The flink's TINYINT and SMALLINT will - // be promoted to - // iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT (backend by 1 - // 'byte'), we will - // read 4 bytes rather than 1 byte, it will mess up the byte array in BinaryRowData. So here - // we must use flink - // schema. + // be promoted to iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT + // (backend by 1 'byte'), we will read 4 bytes rather than 1 byte, it will mess up the + // byte array in BinaryRowData. So here we must use flink schema. return (RowType) requestedSchema.toRowDataType().getLogicalType(); } else { return FlinkSchemaUtil.convert(schema); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.java new file mode 100644 index 000000000000..317fb169ae1b --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.Serializable; +import org.apache.iceberg.io.WriteResult; + +public class FlinkWriteResult implements Serializable { + private final long checkpointId; + private final WriteResult writeResult; + + public FlinkWriteResult(long checkpointId, WriteResult writeResult) { + this.checkpointId = checkpointId; + this.writeResult = writeResult; + } + + public long checkpointId() { + return checkpointId; + } + + public WriteResult writeResult() { + return writeResult; + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java new file mode 100644 index 000000000000..408c3e9a9d5f --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.Objects; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +/** + * The aggregated results of a single checkpoint which should be committed. Containing the + * serialized {@link org.apache.iceberg.flink.sink.DeltaManifests} file - which contains the commit + * data, and the jobId, operatorId, checkpointId triplet which helps identifying the specific commit + * + *

{@link IcebergCommittableSerializer} is used for serializing the objects between the Writer + * and the Aggregator operator and between the Aggregator and the Committer as well. + */ +class IcebergCommittable implements Serializable { + private final byte[] manifest; + private final String jobId; + private final String operatorId; + private final long checkpointId; + + IcebergCommittable(byte[] manifest, String jobId, String operatorId, long checkpointId) { + this.manifest = manifest; + this.jobId = jobId; + this.operatorId = operatorId; + this.checkpointId = checkpointId; + } + + byte[] manifest() { + return manifest; + } + + String jobId() { + return jobId; + } + + String operatorId() { + return operatorId; + } + + Long checkpointId() { + return checkpointId; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("jobId", jobId) + .add("checkpointId", checkpointId) + .add("operatorId", operatorId) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + IcebergCommittable that = (IcebergCommittable) o; + return checkpointId == that.checkpointId + && Arrays.equals(manifest, that.manifest) + && Objects.equals(jobId, that.jobId) + && Objects.equals(operatorId, that.operatorId); + } + + @Override + public int hashCode() { + int result = Objects.hash(jobId, operatorId, checkpointId); + result = 31 * result + Arrays.hashCode(manifest); + return result; + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java new file mode 100644 index 000000000000..e2b388a83c75 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; + +/** + * This serializer is used for serializing the {@link IcebergCommittable} objects between the Writer + * and the Aggregator operator and between the Aggregator and the Committer as well. + * + *

In both cases only the respective part is serialized. + */ +class IcebergCommittableSerializer implements SimpleVersionedSerializer { + private static final int VERSION = 1; + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public byte[] serialize(IcebergCommittable committable) throws IOException { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + DataOutputViewStreamWrapper view = new DataOutputViewStreamWrapper(out); + view.writeUTF(committable.jobId()); + view.writeUTF(committable.operatorId()); + view.writeLong(committable.checkpointId()); + view.writeInt(committable.manifest().length); + view.write(committable.manifest()); + return out.toByteArray(); + } + + @Override + public IcebergCommittable deserialize(int version, byte[] serialized) throws IOException { + if (version == 1) { + DataInputDeserializer view = new DataInputDeserializer(serialized); + String jobId = view.readUTF(); + String operatorId = view.readUTF(); + long checkpointId = view.readLong(); + int manifestLen = view.readInt(); + byte[] manifestBuf; + manifestBuf = new byte[manifestLen]; + view.read(manifestBuf); + return new IcebergCommittable(manifestBuf, jobId, operatorId, checkpointId); + } + throw new IOException("Unrecognized version or corrupt state: " + version); + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java new file mode 100644 index 000000000000..3048b51c6cb0 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java @@ -0,0 +1,312 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ReplacePartitions; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.SnapshotUpdate; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class implements the Flink SinkV2 {@link Committer} interface to implement the Iceberg + * commits. The implementation builds on the following assumptions: + * + *

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

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

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

tableSupplier; + if (tableRefreshInterval != null) { + tableSupplier = new CachingTableSupplier(table, tableLoader(), tableRefreshInterval); + } else { + tableSupplier = () -> serializableTable; + } + + boolean overwriteMode = flinkWriteConf.overwriteMode(); + + // Validate the equality fields and partition fields if we enable the upsert mode. + List equalityFieldIds = + SinkUtil.checkAndGetEqualityFieldIds(table, equalityFieldColumns); + + if (flinkWriteConf.upsertMode()) { + Preconditions.checkState( + !overwriteMode, + "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); + Preconditions.checkState( + !equalityFieldIds.isEmpty(), + "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + if (!table.spec().isUnpartitioned()) { + for (PartitionField partitionField : table.spec().fields()) { + Preconditions.checkState( + equalityFieldIds.contains(partitionField.sourceId()), + "In UPSERT mode, partition field '%s' should be included in equality fields: '%s'", + partitionField, + equalityFieldColumns); + } + } + } + + return new IcebergSink( + tableLoader, + table, + snapshotSummary, + uidSuffix, + writeProperties(table, flinkWriteConf.dataFileFormat(), flinkWriteConf), + toFlinkRowType(table.schema(), tableSchema), + tableSupplier, + flinkWriteConf, + equalityFieldIds, + flinkWriteConf.branch(), + overwriteMode); + } + + /** + * Append the iceberg sink operators to write records to iceberg table. + * + * @return {@link DataStreamSink} for sink. + */ + @Override + public DataStreamSink append() { + IcebergSink sink = build(); + String suffix = defaultSuffix(uidSuffix, table.name()); + DataStream rowDataInput = inputCreator.apply(suffix); + // Please note that V2 sink framework will apply the uid here to the framework created + // operators like writer, + // committer. E.g. "Sink writer: + DataStreamSink rowDataDataStreamSink = + rowDataInput.sinkTo(sink).uid(suffix).name(suffix); + + // Note that IcebergSink internally consists o multiple operators (like writer, committer, + // aggregator). + // The following parallelism will be propagated to all of the above operators. + if (sink.flinkWriteConf.writeParallelism() != null) { + rowDataDataStreamSink.setParallelism(sink.flinkWriteConf.writeParallelism()); + } + return rowDataDataStreamSink; + } + } + + private static String defaultSuffix(String uidSuffix, String defaultSuffix) { + if (uidSuffix == null || uidSuffix.isEmpty()) { + return defaultSuffix; + } + return uidSuffix; + } + + private static SerializableTable checkAndGetTable(TableLoader tableLoader, Table table) { + if (table == null) { + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + try (TableLoader loader = tableLoader) { + return (SerializableTable) SerializableTable.copyOf(loader.loadTable()); + } catch (IOException e) { + throw new UncheckedIOException( + "Failed to load iceberg table from table loader: " + tableLoader, e); + } + } + + return (SerializableTable) SerializableTable.copyOf(table); + } + + private static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) { + if (requestedSchema != null) { + // Convert the flink schema to iceberg schema firstly, then reassign ids to match the existing + // iceberg schema. + Schema writeSchema = TypeUtil.reassignIds(FlinkSchemaUtil.convert(requestedSchema), schema); + TypeUtil.validateWriteSchema(schema, writeSchema, true, true); + + // We use this flink schema to read values from RowData. The flink's TINYINT and SMALLINT will + // be promoted to iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT + // (backend by 1 'byte'), we will read 4 bytes rather than 1 byte, it will mess up the byte + // array in BinaryRowData. So here we must use flink schema. + return (RowType) requestedSchema.toRowDataType().getLogicalType(); + } else { + return FlinkSchemaUtil.convert(schema); + } + } + + /** + * Based on the {@link FileFormat} overwrites the table level compression properties for the table + * write. + * + * @param table The table to get the table level settings + * @param format The FileFormat to use + * @param conf The write configuration + * @return The properties to use for writing + */ + private static Map writeProperties( + Table table, FileFormat format, FlinkWriteConf conf) { + Map writeProperties = Maps.newHashMap(table.properties()); + + switch (format) { + case PARQUET: + writeProperties.put(PARQUET_COMPRESSION, conf.parquetCompressionCodec()); + String parquetCompressionLevel = conf.parquetCompressionLevel(); + if (parquetCompressionLevel != null) { + writeProperties.put(PARQUET_COMPRESSION_LEVEL, parquetCompressionLevel); + } + + break; + case AVRO: + writeProperties.put(AVRO_COMPRESSION, conf.avroCompressionCodec()); + String avroCompressionLevel = conf.avroCompressionLevel(); + if (avroCompressionLevel != null) { + writeProperties.put(AVRO_COMPRESSION_LEVEL, conf.avroCompressionLevel()); + } + + break; + case ORC: + writeProperties.put(ORC_COMPRESSION, conf.orcCompressionCodec()); + writeProperties.put(ORC_COMPRESSION_STRATEGY, conf.orcCompressionStrategy()); + break; + default: + throw new IllegalArgumentException(String.format("Unknown file format %s", format)); + } + + return writeProperties; + } + + private DataStream distributeDataStream(DataStream input) { + DistributionMode mode = flinkWriteConf.distributionMode(); + Schema schema = table.schema(); + PartitionSpec spec = table.spec(); + LOG.info("Write distribution mode is '{}'", mode.modeName()); + switch (mode) { + case NONE: + if (equalityFieldIds.isEmpty()) { + return input; + } else { + LOG.info("Distribute rows by equality fields, because there are equality fields set"); + return input.keyBy(new EqualityFieldKeySelector(schema, flinkRowType, equalityFieldIds)); + } + + case HASH: + if (equalityFieldIds.isEmpty()) { + if (table.spec().isUnpartitioned()) { + LOG.warn( + "Fallback to use 'none' distribution mode, because there are no equality fields set " + + "and table is unpartitioned"); + return input; + } else { + if (BucketPartitionerUtil.hasOneBucketField(spec)) { + return input.partitionCustom( + new BucketPartitioner(spec), + new BucketPartitionKeySelector(spec, schema, flinkRowType)); + } else { + return input.keyBy(new PartitionKeySelector(spec, schema, flinkRowType)); + } + } + } else { + if (spec.isUnpartitioned()) { + LOG.info( + "Distribute rows by equality fields, because there are equality fields set " + + "and table is unpartitioned"); + return input.keyBy( + new EqualityFieldKeySelector(schema, flinkRowType, equalityFieldIds)); + } else { + for (PartitionField partitionField : spec.fields()) { + Preconditions.checkState( + equalityFieldIds.contains(partitionField.sourceId()), + "In 'hash' distribution mode with equality fields set, partition field '%s' " + + "should be included in equality fields: '%s'", + partitionField, + equalityFieldColumns); + } + return input.keyBy(new PartitionKeySelector(spec, schema, flinkRowType)); + } + } + + case RANGE: + if (equalityFieldIds.isEmpty()) { + LOG.warn( + "Fallback to use 'none' distribution mode, because there are no equality fields set " + + "and {}=range is not supported yet in flink", + WRITE_DISTRIBUTION_MODE); + return input; + } else { + LOG.info( + "Distribute rows by equality fields, because there are equality fields set " + + "and{}=range is not supported yet in flink", + WRITE_DISTRIBUTION_MODE); + return input.keyBy(new EqualityFieldKeySelector(schema, flinkRowType, equalityFieldIds)); + } + + default: + throw new RuntimeException("Unrecognized " + WRITE_DISTRIBUTION_MODE + ": " + mode); + } + } + + /** + * Initialize a {@link Builder} to export the data from generic input data stream into iceberg + * table. We use {@link RowData} inside the sink connector, so users need to provide a mapper + * function and a {@link TypeInformation} to convert those generic records to a RowData + * DataStream. + * + * @param input the generic source input data stream. + * @param mapper function to convert the generic data to {@link RowData} + * @param outputType to define the {@link TypeInformation} for the input data. + * @param the data type of records. + * @return {@link Builder} to connect the iceberg table. + */ + public static Builder builderFor( + DataStream input, MapFunction mapper, TypeInformation outputType) { + return new Builder().forMapperOutputType(input, mapper, outputType); + } + + /** + * Initialize a {@link Builder} to export the data from input data stream with {@link Row}s into + * iceberg table. We use {@link RowData} inside the sink connector, so users need to provide a + * {@link TableSchema} for builder to convert those {@link Row}s to a {@link RowData} DataStream. + * + * @param input the source input data stream with {@link Row}s. + * @param tableSchema defines the {@link TypeInformation} for input data. + * @return {@link Builder} to connect the iceberg table. + */ + public static Builder forRow(DataStream input, TableSchema tableSchema) { + return new Builder().forRow(input, tableSchema); + } + + /** + * Initialize a {@link Builder} to export the data from input data stream with {@link RowData}s + * into iceberg table. + * + * @param input the source input data stream with {@link RowData}s. + * @return {@link Builder} to connect the iceberg table. + */ + public static Builder forRowData(DataStream input) { + return new Builder().forRowData(input); + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkBuilder.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkBuilder.java new file mode 100644 index 000000000000..f232df512894 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkBuilder.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.util.List; +import java.util.Map; +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; + +/** + * This class is for internal purpose of transition between the previous implementation of Flink's + * sink ({@link FlinkSink}) and the new one implementation based on Flink SinkV2 API ({@link + * IcebergSink}). After we remove the previous implementation, all occurrences of this class would + * be replaced by direct {@link IcebergSink} usage. + */ +@Internal +interface IcebergSinkBuilder> { + + T tableSchema(TableSchema newTableSchema); + + T tableLoader(TableLoader newTableLoader); + + T equalityFieldColumns(List columns); + + T overwrite(boolean newOverwrite); + + T setAll(Map properties); + + T flinkConf(ReadableConfig config); + + T table(Table newTable); + + T writeParallelism(int newWriteParallelism); + + T distributionMode(DistributionMode mode); + + T toBranch(String branch); + + T upsert(boolean enabled); + + DataStreamSink append(); + + static IcebergSinkBuilder forRow( + DataStream input, TableSchema tableSchema, boolean useV2Sink) { + if (useV2Sink) { + return IcebergSink.forRow(input, tableSchema); + } else { + return FlinkSink.forRow(input, tableSchema); + } + } + + static IcebergSinkBuilder forRowData(DataStream input, boolean useV2Sink) { + if (useV2Sink) { + return IcebergSink.forRowData(input); + } else { + return FlinkSink.forRowData(input); + } + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkWriter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkWriter.java new file mode 100644 index 000000000000..7234cf74020e --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkWriter.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.IOException; +import java.util.Collection; +import java.util.concurrent.TimeUnit; +import org.apache.flink.api.connector.sink2.CommittingSinkWriter; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Iceberg writer implementation for the {@link SinkWriter} interface. Used by the {@link + * org.apache.iceberg.flink.sink.IcebergSink} (SinkV2). Writes out the data to the final place, and + * emits a single {@link WriteResult} at every checkpoint for every data/delete file created by this + * writer. + */ +class IcebergSinkWriter implements CommittingSinkWriter { + private static final Logger LOG = LoggerFactory.getLogger(IcebergSinkWriter.class); + + private final String fullTableName; + private final TaskWriterFactory taskWriterFactory; + private final IcebergStreamWriterMetrics metrics; + private TaskWriter writer; + private final int subTaskId; + private final int attemptId; + + IcebergSinkWriter( + String fullTableName, + TaskWriterFactory taskWriterFactory, + IcebergStreamWriterMetrics metrics, + int subTaskId, + int attemptId) { + this.fullTableName = fullTableName; + this.taskWriterFactory = taskWriterFactory; + // Initialize the task writer factory. + taskWriterFactory.initialize(subTaskId, attemptId); + // Initialize the task writer. + this.writer = taskWriterFactory.create(); + this.metrics = metrics; + this.subTaskId = subTaskId; + this.attemptId = attemptId; + LOG.debug( + "Created Stream Writer for table {} subtask {} attemptId {}", + fullTableName, + subTaskId, + attemptId); + } + + @Override + public void write(RowData element, Context context) throws IOException, InterruptedException { + writer.write(element); + } + + @Override + public void flush(boolean endOfInput) { + // flush is used to handle flush/endOfInput, so no action is taken here. + } + + @Override + public void close() throws Exception { + if (writer != null) { + writer.close(); + } + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("tableName", fullTableName) + .add("subTaskId", subTaskId) + .add("attemptId", attemptId) + .toString(); + } + + @Override + public Collection prepareCommit() throws IOException { + long startNano = System.nanoTime(); + WriteResult result = writer.complete(); + this.writer = taskWriterFactory.create(); + metrics.updateFlushResult(result); + metrics.flushDuration(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); + LOG.debug( + "Iceberg writer subtask {} attempt {} flushed {} data files and {} delete files", + subTaskId, + attemptId, + result.dataFiles().length, + result.deleteFiles().length); + return Lists.newArrayList(result); + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java index 9ea0349fb057..412d6c7081bf 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java @@ -29,10 +29,11 @@ import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -class IcebergStreamWriter extends AbstractStreamOperator - implements OneInputStreamOperator, BoundedOneInput { +class IcebergStreamWriter extends AbstractStreamOperator + implements OneInputStreamOperator, BoundedOneInput { private static final long serialVersionUID = 1L; + static final long END_INPUT_CHECKPOINT_ID = Long.MAX_VALUE; private final String fullTableName; private final TaskWriterFactory taskWriterFactory; @@ -63,7 +64,7 @@ public void open() { @Override public void prepareSnapshotPreBarrier(long checkpointId) throws Exception { - flush(); + flush(checkpointId); this.writer = taskWriterFactory.create(); } @@ -89,20 +90,20 @@ public void endInput() throws IOException { // Note that if the task is not closed after calling endInput, checkpoint may be triggered again // causing files to be sent repeatedly, the writer is marked as null after the last file is sent // to guard against duplicated writes. - flush(); + flush(END_INPUT_CHECKPOINT_ID); } @Override public String toString() { return MoreObjects.toStringHelper(this) - .add("table_name", fullTableName) - .add("subtask_id", subTaskId) - .add("attempt_id", attemptId) + .add("tableName", fullTableName) + .add("subTaskId", subTaskId) + .add("attemptId", attemptId) .toString(); } /** close all open files and emit files to downstream committer operator */ - private void flush() throws IOException { + private void flush(long checkpointId) throws IOException { if (writer == null) { return; } @@ -110,7 +111,7 @@ private void flush() throws IOException { long startNano = System.nanoTime(); WriteResult result = writer.complete(); writerMetrics.updateFlushResult(result); - output.collect(new StreamRecord<>(result)); + output.collect(new StreamRecord<>(new FlinkWriteResult(checkpointId, result))); writerMetrics.flushDuration(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); // Set writer to null to prevent duplicate flushes in the corner case of diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java new file mode 100644 index 000000000000..794ade577976 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.IOException; +import java.util.Collection; +import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableSummary; +import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Operator which aggregates the individual {@link WriteResult} objects) to a single {@link + * IcebergCommittable} per checkpoint (storing the serialized {@link + * org.apache.iceberg.flink.sink.DeltaManifests}, jobId, operatorId, checkpointId) + */ +class IcebergWriteAggregator extends AbstractStreamOperator> + implements OneInputStreamOperator< + CommittableMessage, CommittableMessage> { + private static final Logger LOG = LoggerFactory.getLogger(IcebergWriteAggregator.class); + private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; + private final Collection results; + private transient ManifestOutputFileFactory icebergManifestOutputFileFactory; + private transient Table table; + private final TableLoader tableLoader; + + IcebergWriteAggregator(TableLoader tableLoader) { + this.results = Sets.newHashSet(); + this.tableLoader = tableLoader; + } + + @Override + public void open() throws Exception { + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + String flinkJobId = getContainingTask().getEnvironment().getJobID().toString(); + String operatorId = getOperatorID().toString(); + int subTaskId = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(); + Preconditions.checkArgument( + subTaskId == 0, "The subTaskId must be zero in the IcebergWriteAggregator"); + int attemptId = getRuntimeContext().getTaskInfo().getAttemptNumber(); + this.table = tableLoader.loadTable(); + + this.icebergManifestOutputFileFactory = + FlinkManifestUtil.createOutputFileFactory( + () -> table, table.properties(), flinkJobId, operatorId, subTaskId, attemptId); + } + + @Override + public void finish() throws IOException { + prepareSnapshotPreBarrier(Long.MAX_VALUE); + } + + @Override + public void prepareSnapshotPreBarrier(long checkpointId) throws IOException { + IcebergCommittable committable = + new IcebergCommittable( + writeToManifest(results, checkpointId), + getContainingTask().getEnvironment().getJobID().toString(), + getRuntimeContext().getOperatorUniqueID(), + checkpointId); + CommittableMessage summary = + new CommittableSummary<>(0, 1, checkpointId, 1, 1, 0); + output.collect(new StreamRecord<>(summary)); + CommittableMessage message = + new CommittableWithLineage<>(committable, checkpointId, 0); + output.collect(new StreamRecord<>(message)); + LOG.info("Emitted commit message to downstream committer operator"); + results.clear(); + } + + /** + * Write all the completed data files to a newly created manifest file and return the manifest's + * avro serialized bytes. + */ + public byte[] writeToManifest(Collection writeResults, long checkpointId) + throws IOException { + if (writeResults.isEmpty()) { + return EMPTY_MANIFEST_DATA; + } + + WriteResult result = WriteResult.builder().addAll(writeResults).build(); + DeltaManifests deltaManifests = + FlinkManifestUtil.writeCompletedFiles( + result, () -> icebergManifestOutputFileFactory.create(checkpointId), table.spec()); + + return SimpleVersionedSerialization.writeVersionAndSerialize( + DeltaManifestsSerializer.INSTANCE, deltaManifests); + } + + @Override + public void processElement(StreamRecord> element) + throws Exception { + + if (element.isRecord() && element.getValue() instanceof CommittableWithLineage) { + results.add(((CommittableWithLineage) element.getValue()).getCommittable()); + } + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java index da5e6e7627ae..30517cd38216 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink; +import java.util.Locale; import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Supplier; @@ -26,13 +27,13 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableOperations; import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Strings; class ManifestOutputFileFactory { // Users could define their own flink manifests directory by setting this value in table // properties. - static final String FLINK_MANIFEST_LOCATION = "flink.manifests.location"; - + @VisibleForTesting static final String FLINK_MANIFEST_LOCATION = "flink.manifests.location"; private final Supplier
tableSupplier; private final Map props; private final String flinkJobId; @@ -59,6 +60,7 @@ class ManifestOutputFileFactory { private String generatePath(long checkpointId) { return FileFormat.AVRO.addExtension( String.format( + Locale.ROOT, "%s-%s-%05d-%d-%d-%05d", flinkJobId, operatorUniqueId, diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java new file mode 100644 index 000000000000..7f28a50ecaa8 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class SinkUtil { + + private static final long INITIAL_CHECKPOINT_ID = -1L; + + public static final String FLINK_JOB_ID = "flink.job-id"; + + public static final String OPERATOR_ID = "flink.operator-id"; + public static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed-checkpoint-id"; + + private SinkUtil() {} + + private static final Logger LOG = LoggerFactory.getLogger(SinkUtil.class); + + static List checkAndGetEqualityFieldIds(Table table, List equalityFieldColumns) { + List equalityFieldIds = Lists.newArrayList(table.schema().identifierFieldIds()); + if (equalityFieldColumns != null && !equalityFieldColumns.isEmpty()) { + Set equalityFieldSet = Sets.newHashSetWithExpectedSize(equalityFieldColumns.size()); + for (String column : equalityFieldColumns) { + org.apache.iceberg.types.Types.NestedField field = table.schema().findField(column); + Preconditions.checkNotNull( + field, + "Missing required equality field column '%s' in table schema %s", + column, + table.schema()); + equalityFieldSet.add(field.fieldId()); + } + + if (!equalityFieldSet.equals(table.schema().identifierFieldIds())) { + LOG.warn( + "The configured equality field column IDs {} are not matched with the schema identifier field IDs" + + " {}, use job specified equality field columns as the equality fields by default.", + equalityFieldSet, + table.schema().identifierFieldIds()); + } + equalityFieldIds = Lists.newArrayList(equalityFieldSet); + } + return equalityFieldIds; + } + + static long getMaxCommittedCheckpointId( + Table table, String flinkJobId, String operatorId, String branch) { + Snapshot snapshot = table.snapshot(branch); + long lastCommittedCheckpointId = INITIAL_CHECKPOINT_ID; + + while (snapshot != null) { + Map summary = snapshot.summary(); + String snapshotFlinkJobId = summary.get(FLINK_JOB_ID); + String snapshotOperatorId = summary.get(OPERATOR_ID); + if (flinkJobId.equals(snapshotFlinkJobId) + && (snapshotOperatorId == null || snapshotOperatorId.equals(operatorId))) { + String value = summary.get(MAX_COMMITTED_CHECKPOINT_ID); + if (value != null) { + lastCommittedCheckpointId = Long.parseLong(value); + break; + } + } + Long parentSnapshotId = snapshot.parentId(); + snapshot = parentSnapshotId != null ? table.snapshot(parentSnapshotId) : null; + } + + return lastCommittedCheckpointId; + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java new file mode 100644 index 000000000000..5a44373cccaa --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.util.InstantiationUtil; +import org.apache.iceberg.io.WriteResult; + +class WriteResultSerializer implements SimpleVersionedSerializer { + private static final int VERSION = 1; + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public byte[] serialize(WriteResult writeResult) throws IOException { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + DataOutputViewStreamWrapper view = new DataOutputViewStreamWrapper(out); + byte[] result = InstantiationUtil.serializeObject(writeResult); + view.write(result); + return out.toByteArray(); + } + + @Override + public WriteResult deserialize(int version, byte[] serialized) throws IOException { + if (version == 1) { + DataInputDeserializer view = new DataInputDeserializer(serialized); + byte[] resultBuf = new byte[serialized.length]; + view.read(resultBuf); + try { + return InstantiationUtil.deserializeObject( + resultBuf, IcebergCommittableSerializer.class.getClassLoader()); + } catch (ClassNotFoundException cnc) { + throw new IOException("Could not deserialize the WriteResult object", cnc); + } + } + throw new IOException("Unrecognized version or corrupt state: " + version); + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java index 5525f02c873e..95c2328f032a 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java @@ -186,7 +186,9 @@ private boolean isComplete() { return subtaskSet.size() == parallelism; } - /** @return false if duplicate */ + /** + * @return false if duplicate + */ private boolean merge(int subtask, DataStatistics taskStatistics) { if (subtaskSet.contains(subtask)) { return false; diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java index 4bfde7204acf..e2a282efd82e 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.sink.shuffle; import java.util.Comparator; +import java.util.Locale; import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; @@ -265,8 +266,10 @@ private void sendGlobalStatisticsToSubtasks(GlobalStatistics statistics) { } }, String.format( + Locale.ROOT, "Failed to send operator %s coordinator global data statistics for checkpoint %d", - operatorName, statistics.checkpointId())); + operatorName, + statistics.checkpointId())); } @SuppressWarnings("FutureReturnValueIgnored") @@ -290,8 +293,11 @@ private void handleRequestGlobalStatisticsEvent(int subtask, RequestGlobalStatis } }, String.format( + Locale.ROOT, "Failed to send operator %s coordinator global data statistics to requesting subtask %d for checkpoint %d", - operatorName, subtask, globalStatistics.checkpointId())); + operatorName, + subtask, + globalStatistics.checkpointId())); } else { LOG.info( "Ignore global statistics request from subtask {} as statistics not available", subtask); @@ -318,8 +324,11 @@ public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEven } }, String.format( + Locale.ROOT, "handling operator event %s from subtask %d (#%d)", - event.getClass(), subtask, attemptNumber)); + event.getClass(), + subtask, + attemptNumber)); } @Override @@ -339,7 +348,7 @@ public void checkpointCoordinator(long checkpointId, CompletableFuture r completedStatistics, completedStatisticsSerializer)); } }, - String.format("taking checkpoint %d", checkpointId)); + String.format(Locale.ROOT, "taking checkpoint %d", checkpointId)); } @Override @@ -381,7 +390,8 @@ public void subtaskReset(int subtask, long checkpointId) { this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); subtaskGateways.reset(subtask); }, - String.format("handling subtask %d recovery to checkpoint %d", subtask, checkpointId)); + String.format( + Locale.ROOT, "handling subtask %d recovery to checkpoint %d", subtask, checkpointId)); } @Override @@ -397,7 +407,7 @@ public void executionAttemptFailed(int subtask, int attemptNumber, @Nullable Thr this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); subtaskGateways.unregisterSubtaskGateway(subtask, attemptNumber); }, - String.format("handling subtask %d (#%d) failure", subtask, attemptNumber)); + String.format(Locale.ROOT, "handling subtask %d (#%d) failure", subtask, attemptNumber)); } @Override @@ -411,7 +421,10 @@ public void executionAttemptReady(int subtask, int attemptNumber, SubtaskGateway subtaskGateways.registerSubtaskGateway(gateway); }, String.format( - "making event gateway to subtask %d (#%d) available", subtask, attemptNumber)); + Locale.ROOT, + "making event gateway to subtask %d (#%d) available", + subtask, + attemptNumber)); } @VisibleForTesting diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java new file mode 100644 index 000000000000..dc147bf36d13 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; + +@Internal +public class DataStatisticsOperatorFactory extends AbstractStreamOperatorFactory + implements CoordinatedOperatorFactory, + OneInputStreamOperatorFactory { + + private final Schema schema; + private final SortOrder sortOrder; + private final int downstreamParallelism; + private final StatisticsType type; + private final double closeFileCostWeightPercentage; + + public DataStatisticsOperatorFactory( + Schema schema, + SortOrder sortOrder, + int downstreamParallelism, + StatisticsType type, + double closeFileCostWeightPercentage) { + this.schema = schema; + this.sortOrder = sortOrder; + this.downstreamParallelism = downstreamParallelism; + this.type = type; + this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; + } + + @Override + public OperatorCoordinator.Provider getCoordinatorProvider( + String operatorName, OperatorID operatorID) { + return new DataStatisticsCoordinatorProvider( + operatorName, + operatorID, + schema, + sortOrder, + downstreamParallelism, + type, + closeFileCostWeightPercentage); + } + + @SuppressWarnings("unchecked") + @Override + public > T createStreamOperator( + StreamOperatorParameters parameters) { + OperatorID operatorId = parameters.getStreamConfig().getOperatorID(); + String operatorName = parameters.getStreamConfig().getOperatorName(); + OperatorEventGateway gateway = + parameters.getOperatorEventDispatcher().getOperatorEventGateway(operatorId); + + DataStatisticsOperator rangeStatisticsOperator = + new DataStatisticsOperator( + operatorName, schema, sortOrder, gateway, downstreamParallelism, type); + + rangeStatisticsOperator.setup( + parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); + parameters + .getOperatorEventDispatcher() + .registerEventHandler(operatorId, rangeStatisticsOperator); + + return (T) rangeStatisticsOperator; + } + + @SuppressWarnings("rawtypes") + @Override + public Class getStreamOperatorClass(ClassLoader classLoader) { + return DataStatisticsOperator.class; + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java index ce17e1964392..ce94bec14860 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java @@ -27,7 +27,9 @@ class RequestGlobalStatisticsEvent implements OperatorEvent { this.signature = null; } - /** @param signature hashCode of the subtask's existing global statistics */ + /** + * @param signature hashCode of the subtask's existing global statistics + */ RequestGlobalStatisticsEvent(int signature) { this.signature = signature; } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java index b82fc8250763..b3e536bdde52 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java @@ -23,6 +23,7 @@ import java.math.BigInteger; import java.nio.ByteBuffer; import java.util.List; +import java.util.Locale; import java.util.Objects; import java.util.Set; import java.util.UUID; @@ -169,7 +170,8 @@ public void serialize(SortKey record, DataOutputView target) throws IOException default: // SortKey transformation is a flattened struct without list and map throw new UnsupportedOperationException( - String.format("Field %d has unsupported field type: %s", fieldId, typeId)); + String.format( + Locale.ROOT, "Field %d has unsupported field type: %s", fieldId, typeId)); } } } @@ -238,7 +240,8 @@ public SortKey deserialize(SortKey reuse, DataInputView source) throws IOExcepti default: // SortKey transformation is a flattened struct without list and map throw new UnsupportedOperationException( - String.format("Field %d has unsupported field type: %s", fieldId, typeId)); + String.format( + Locale.ROOT, "Field %d has unsupported field type: %s", fieldId, typeId)); } } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java index 91d975349b19..3beda960cec8 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.util.Iterator; +import java.util.Locale; import org.apache.flink.annotation.Internal; import org.apache.iceberg.CombinedScanTask; import org.apache.iceberg.FileScanTask; @@ -95,8 +96,11 @@ public void seek(int startingFileOffset, long startingRecordOffset) { } else { throw new IllegalStateException( String.format( + Locale.ROOT, "Invalid starting record offset %d for file %d from CombinedScanTask: %s", - startingRecordOffset, startingFileOffset, combinedTask)); + startingRecordOffset, + startingFileOffset, + combinedTask)); } } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java index 9a5123dc489e..a68f0e50e0d0 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java @@ -93,7 +93,7 @@ public FlinkInputSplit[] createInputSplits(int minNumSplits) throws IOException // Called in Job manager, so it is OK to load table from catalog. tableLoader.open(); final ExecutorService workerPool = - ThreadPools.newWorkerPool("iceberg-plan-worker-pool", context.planParallelism()); + ThreadPools.newFixedThreadPool("iceberg-plan-worker-pool", context.planParallelism()); try (TableLoader loader = tableLoader) { Table table = loader.loadTable(); return FlinkSplitPlanner.planInputSplits(table, context, workerPool); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java index b1431a32dd20..5dcb4b8cf988 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java @@ -28,6 +28,7 @@ import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.iceberg.Schema; @@ -46,6 +47,14 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.PropertyUtil; +/** + * /** Flink source builder for old {@link SourceFunction} implementation. + * + * @deprecated since 1.7.0, will be removed in 2.0.0. Use {@link IcebergSource} instead, which + * implement the newer FLIP-27 source interface. This class implements the old {@link + * SourceFunction} that has been marked as deprecated in Flink since Aug 2023. + */ +@Deprecated public class FlinkSource { private FlinkSource() {} @@ -124,7 +133,9 @@ public Builder setAll(Map properties) { return this; } - /** @deprecated Use {@link #setAll} instead. */ + /** + * @deprecated Use {@link #setAll} instead. + */ @Deprecated public Builder properties(Map properties) { readOptions.putAll(properties); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index ccbd0d9997ed..be29d4b52b69 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -27,7 +27,8 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; -import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.Source; import org.apache.flink.api.connector.source.SourceReader; @@ -37,6 +38,9 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.flink.util.Preconditions; @@ -61,10 +65,12 @@ import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer; import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator; import org.apache.iceberg.flink.source.reader.ColumnStatsWatermarkExtractor; +import org.apache.iceberg.flink.source.reader.ConverterReaderFunction; import org.apache.iceberg.flink.source.reader.IcebergSourceReader; import org.apache.iceberg.flink.source.reader.IcebergSourceReaderMetrics; import org.apache.iceberg.flink.source.reader.MetaDataReaderFunction; import org.apache.iceberg.flink.source.reader.ReaderFunction; +import org.apache.iceberg.flink.source.reader.RowDataConverter; import org.apache.iceberg.flink.source.reader.RowDataReaderFunction; import org.apache.iceberg.flink.source.reader.SerializableRecordEmitter; import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; @@ -72,13 +78,13 @@ import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.flink.source.split.SplitComparators; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.ThreadPools; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -@Experimental public class IcebergSource implements Source { private static final Logger LOG = LoggerFactory.getLogger(IcebergSource.class); @@ -95,6 +101,11 @@ public class IcebergSource implements Source emitter; private final String tableName; + // cache the discovered splits by planSplitsForBatch, which can be called twice. And they come + // from two different threads: (1) source/stream construction by main thread (2) enumerator + // creation. Hence need volatile here. + private volatile List batchSplits; + IcebergSource( TableLoader tableLoader, ScanContext scanContext, @@ -130,16 +141,26 @@ private String planningThreadName() { return tableName + "-" + UUID.randomUUID(); } + /** + * Cache the enumerated splits for batch execution to avoid double planning as there are two code + * paths obtaining splits: (1) infer parallelism (2) enumerator creation. + */ private List planSplitsForBatch(String threadName) { + if (batchSplits != null) { + return batchSplits; + } + ExecutorService workerPool = - ThreadPools.newWorkerPool(threadName, scanContext.planParallelism()); + ThreadPools.newFixedThreadPool(threadName, scanContext.planParallelism()); try (TableLoader loader = tableLoader.clone()) { loader.open(); - List splits = + this.batchSplits = FlinkSplitPlanner.planIcebergSourceSplits(loader.loadTable(), scanContext, workerPool); LOG.info( - "Discovered {} splits from table {} during job initialization", splits.size(), tableName); - return splits; + "Discovered {} splits from table {} during job initialization", + batchSplits.size(), + tableName); + return batchSplits; } catch (IOException e) { throw new UncheckedIOException("Failed to close table loader", e); } finally { @@ -205,26 +226,69 @@ private SplitEnumerator createEnumer // Only do scan planning if nothing is restored from checkpoint state List splits = planSplitsForBatch(planningThreadName()); assigner.onDiscoveredSplits(splits); + // clear the cached splits after enumerator creation as they won't be needed anymore + this.batchSplits = null; } return new StaticIcebergEnumerator(enumContext, assigner); } } + private boolean shouldInferParallelism() { + return !scanContext.isStreaming(); + } + + private int inferParallelism(ReadableConfig flinkConf, StreamExecutionEnvironment env) { + int parallelism = + SourceUtil.inferParallelism( + flinkConf, + scanContext.limit(), + () -> { + List splits = planSplitsForBatch(planningThreadName()); + return splits.size(); + }); + + if (env.getMaxParallelism() > 0) { + parallelism = Math.min(parallelism, env.getMaxParallelism()); + } + + return parallelism; + } + + /** + * Create a source builder. + * + * @deprecated since 1.7.0. Will be removed in 2.0.0; use{@link IcebergSource#forRowData()} or + * {@link IcebergSource#forOutputType(RowDataConverter)} instead + */ + @Deprecated public static Builder builder() { return new Builder<>(); } + /** Create a source builder for RowData output type. */ public static Builder forRowData() { return new Builder<>(); } + /** + * Create a source builder that would convert {@link RowData} to the output type {@code T}. + * + * @param converter convert {@link RowData} to output type {@code T} + * @param output type + * @return an IcebergSource builder + */ + public static Builder forOutputType(RowDataConverter converter) { + return new Builder().converter(converter); + } + public static class Builder { private TableLoader tableLoader; private Table table; private SplitAssignerFactory splitAssignerFactory; private SerializableComparator splitComparator; private ReaderFunction readerFunction; + private RowDataConverter converter; private ReadableConfig flinkConfig = new Configuration(); private final ScanContext.Builder contextBuilder = ScanContext.builder(); private TableSchema projectedFlinkSchema; @@ -255,11 +319,28 @@ public Builder splitComparator( return this; } + /** + * @deprecated since 1.7.0. Will be removed in 2.0.0; use{@link + * IcebergSource#forOutputType(RowDataConverter)} instead to produce output type other than + * {@link RowData}. + */ + @Deprecated public Builder readerFunction(ReaderFunction newReaderFunction) { + Preconditions.checkState( + converter == null, + "Cannot set reader function when builder was created via IcebergSource.forOutputType(Converter)"); this.readerFunction = newReaderFunction; return this; } + /** + * Don't need to be public. It is set by {@link IcebergSource#forOutputType(RowDataConverter)}. + */ + private Builder converter(RowDataConverter newConverter) { + this.converter = newConverter; + return this; + } + public Builder flinkConfig(ReadableConfig config) { this.flinkConfig = config; return this; @@ -464,7 +545,9 @@ public Builder watermarkColumnTimeUnit(TimeUnit timeUnit) { return this; } - /** @deprecated Use {@link #setAll} instead. */ + /** + * @deprecated Use {@link #setAll} instead. + */ @Deprecated public Builder properties(Map properties) { readOptions.putAll(properties); @@ -482,6 +565,10 @@ public IcebergSource build() { } contextBuilder.resolveConfig(table, readOptions, flinkConfig); + contextBuilder.exposeLocality( + SourceUtil.isLocalityEnabled(table, flinkConfig, exposeLocality)); + contextBuilder.planParallelism( + flinkConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE)); Schema icebergSchema = table.schema(); if (projectedFlinkSchema != null) { contextBuilder.project(FlinkSchemaUtil.convert(icebergSchema, projectedFlinkSchema)); @@ -506,25 +593,7 @@ public IcebergSource build() { ScanContext context = contextBuilder.build(); context.validate(); if (readerFunction == null) { - if (table instanceof BaseMetadataTable) { - MetaDataReaderFunction rowDataReaderFunction = - new MetaDataReaderFunction( - flinkConfig, table.schema(), context.project(), table.io(), table.encryption()); - this.readerFunction = (ReaderFunction) rowDataReaderFunction; - } else { - RowDataReaderFunction rowDataReaderFunction = - new RowDataReaderFunction( - flinkConfig, - table.schema(), - context.project(), - context.nameMapping(), - context.caseSensitive(), - table.io(), - table.encryption(), - context.filters(), - context.limit()); - this.readerFunction = (ReaderFunction) rowDataReaderFunction; - } + this.readerFunction = readerFunction(context); } if (splitAssignerFactory == null) { @@ -545,5 +614,75 @@ public IcebergSource build() { table, emitter); } + + /** + * Build the {@link IcebergSource} and create a {@link DataStream} from the source. Watermark + * strategy is set to {@link WatermarkStrategy#noWatermarks()}. + * + * @return data stream from the Iceberg source + */ + public DataStream buildStream(StreamExecutionEnvironment env) { + // buildStream should only be called with RowData or Converter paths. + Preconditions.checkState( + readerFunction == null, + "Cannot set reader function when building a data stream from the source"); + IcebergSource source = build(); + TypeInformation outputTypeInfo = + outputTypeInfo(converter, table.schema(), source.scanContext.project()); + DataStreamSource stream = + env.fromSource(source, WatermarkStrategy.noWatermarks(), source.name(), outputTypeInfo); + if (source.shouldInferParallelism()) { + stream = stream.setParallelism(source.inferParallelism(flinkConfig, env)); + } + + return stream; + } + + private static TypeInformation outputTypeInfo( + RowDataConverter converter, Schema tableSchema, Schema projected) { + if (converter != null) { + return converter.getProducedType(); + } else { + // output type is RowData + Schema readSchema = projected != null ? projected : tableSchema; + return (TypeInformation) + FlinkCompatibilityUtil.toTypeInfo(FlinkSchemaUtil.convert(readSchema)); + } + } + + private ReaderFunction readerFunction(ScanContext context) { + if (table instanceof BaseMetadataTable) { + MetaDataReaderFunction rowDataReaderFunction = + new MetaDataReaderFunction( + flinkConfig, table.schema(), context.project(), table.io(), table.encryption()); + return (ReaderFunction) rowDataReaderFunction; + } else { + if (converter == null) { + return (ReaderFunction) + new RowDataReaderFunction( + flinkConfig, + table.schema(), + context.project(), + context.nameMapping(), + context.caseSensitive(), + table.io(), + table.encryption(), + context.filters(), + context.limit()); + } else { + return new ConverterReaderFunction<>( + converter, + flinkConfig, + table.schema(), + context.project(), + context.nameMapping(), + context.caseSensitive(), + table.io(), + table.encryption(), + context.filters(), + context.limit()); + } + } + } } } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java index 610657e8d47b..65adce77d9f9 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java @@ -23,11 +23,8 @@ import java.util.Map; import java.util.Optional; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.connector.ChangelogMode; @@ -128,26 +125,18 @@ private DataStream createDataStream(StreamExecutionEnvironment execEnv) .build(); } - private DataStreamSource createFLIP27Stream(StreamExecutionEnvironment env) { + private DataStream createFLIP27Stream(StreamExecutionEnvironment env) { SplitAssignerType assignerType = readableConfig.get(FlinkConfigOptions.TABLE_EXEC_SPLIT_ASSIGNER_TYPE); - IcebergSource source = - IcebergSource.forRowData() - .tableLoader(loader) - .assignerFactory(assignerType.factory()) - .properties(properties) - .project(getProjectedSchema()) - .limit(limit) - .filters(filters) - .flinkConfig(readableConfig) - .build(); - DataStreamSource stream = - env.fromSource( - source, - WatermarkStrategy.noWatermarks(), - source.name(), - TypeInformation.of(RowData.class)); - return stream; + return IcebergSource.forRowData() + .tableLoader(loader) + .assignerFactory(assignerType.factory()) + .properties(properties) + .project(getProjectedSchema()) + .limit(limit) + .filters(filters) + .flinkConfig(readableConfig) + .buildStream(env); } private TableSchema getProjectedSchema() { diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java index c958604c004a..391633924264 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java @@ -104,10 +104,7 @@ public static class RewriteMap extends RichMapFunction taskWriterFactory; private final RowDataFileScanTaskReader rowDataReader; @@ -119,10 +116,7 @@ public RewriteMap( boolean caseSensitive, EncryptionManager encryptionManager, TaskWriterFactory taskWriterFactory) { - this.schema = schema; - this.nameMapping = nameMapping; this.io = io; - this.caseSensitive = caseSensitive; this.encryptionManager = encryptionManager; this.taskWriterFactory = taskWriterFactory; this.rowDataReader = diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java index a07613aee59b..39f615aeacc5 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java @@ -106,7 +106,7 @@ public void open(Configuration parameters) throws Exception { "context should be instance of StreamingRuntimeContext"); final String operatorID = ((StreamingRuntimeContext) runtimeContext).getOperatorUniqueID(); this.workerPool = - ThreadPools.newWorkerPool( + ThreadPools.newFixedThreadPool( "iceberg-worker-pool-" + operatorID, scanContext.planParallelism()); } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java index 280a126a46ce..fc310606dee9 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java @@ -22,6 +22,7 @@ import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicReference; @@ -76,6 +77,7 @@ public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname // Iceberg source uses custom split request event to piggyback finished split ids. throw new UnsupportedOperationException( String.format( + Locale.ROOT, "Received invalid default split request event " + "from subtask %d as Iceberg source uses custom split request event", subtaskId)); @@ -92,8 +94,10 @@ public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { } else { throw new IllegalArgumentException( String.format( + Locale.ROOT, "Received unknown event from subtask %d: %s", - subtaskId, sourceEvent.getClass().getCanonicalName())); + subtaskId, + sourceEvent.getClass().getCanonicalName())); } } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java index c7021b9c6847..c50c3854ee14 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java @@ -37,6 +37,7 @@ public class ContinuousIcebergEnumerator extends AbstractIcebergEnumerator { private static final Logger LOG = LoggerFactory.getLogger(ContinuousIcebergEnumerator.class); + /** * This is hardcoded, as {@link ScanContext#maxPlanningSnapshotCount()} could be the knob to * control the total number of snapshots worth of splits tracked by assigner. diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java index fef4ec45ed8a..9c99d442174f 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java @@ -64,7 +64,7 @@ public ContinuousSplitPlannerImpl( this.workerPool = isSharedPool ? ThreadPools.getWorkerPool() - : ThreadPools.newWorkerPool( + : ThreadPools.newFixedThreadPool( "iceberg-plan-worker-pool-" + threadName, scanContext.planParallelism()); } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.java new file mode 100644 index 000000000000..b158b0871a53 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.avro.RowDataToAvroConverters; +import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; +import org.apache.flink.formats.avro.typeutils.GenericRecordAvroTypeInfo; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.flink.FlinkSchemaUtil; + +public class AvroGenericRecordConverter implements RowDataConverter { + private final Schema avroSchema; + private final RowDataToAvroConverters.RowDataToAvroConverter flinkConverter; + private final TypeInformation outputTypeInfo; + + private AvroGenericRecordConverter(Schema avroSchema, RowType rowType) { + this.avroSchema = avroSchema; + this.flinkConverter = RowDataToAvroConverters.createConverter(rowType); + this.outputTypeInfo = new GenericRecordAvroTypeInfo(avroSchema); + } + + public static AvroGenericRecordConverter fromIcebergSchema( + org.apache.iceberg.Schema icebergSchema, String tableName) { + RowType rowType = FlinkSchemaUtil.convert(icebergSchema); + Schema avroSchema = AvroSchemaUtil.convert(icebergSchema, tableName); + return new AvroGenericRecordConverter(avroSchema, rowType); + } + + public static AvroGenericRecordConverter fromAvroSchema(Schema avroSchema, String tableName) { + DataType dataType = AvroSchemaConverter.convertToDataType(avroSchema.toString()); + LogicalType logicalType = TypeConversions.fromDataToLogicalType(dataType); + RowType rowType = RowType.of(logicalType.getChildren().toArray(new LogicalType[0])); + return new AvroGenericRecordConverter(avroSchema, rowType); + } + + @Override + public GenericRecord apply(RowData rowData) { + return (GenericRecord) flinkConverter.convert(avroSchema, rowData); + } + + @Override + public TypeInformation getProducedType() { + return outputTypeInfo; + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java index 66e59633fff2..f89e5ce13474 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java @@ -28,13 +28,21 @@ import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.flink.source.AvroGenericRecordFileScanTaskReader; import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.IcebergSource; import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; import org.apache.iceberg.flink.source.RowDataToAvroGenericRecordConverter; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -/** Read Iceberg rows as {@link GenericRecord}. */ +/** + * Read Iceberg rows as {@link GenericRecord}. + * + * @deprecated since 1.7.0. Will be removed in 2.0.0; use {@link + * IcebergSource#forOutputType(RowDataConverter)} and {@link AvroGenericRecordConverter} + * instead. + */ +@Deprecated public class AvroGenericRecordReaderFunction extends DataIteratorReaderFunction { private final String tableName; private final Schema readSchema; diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java new file mode 100644 index 000000000000..e1e7c17d63c5 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import java.util.List; +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Schema; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.InputFilesDecryptor; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.FileScanTaskReader; +import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +@Internal +public class ConverterReaderFunction extends DataIteratorReaderFunction { + private final RowDataConverter converter; + private final Schema tableSchema; + private final Schema readSchema; + private final String nameMapping; + private final boolean caseSensitive; + private final FileIO io; + private final EncryptionManager encryption; + private final List filters; + private final long limit; + + private transient RecordLimiter recordLimiter = null; + + public ConverterReaderFunction( + RowDataConverter converter, + ReadableConfig config, + Schema tableSchema, + Schema projectedSchema, + String nameMapping, + boolean caseSensitive, + FileIO io, + EncryptionManager encryption, + List filters, + long limit) { + super(new ListDataIteratorBatcher<>(config)); + this.converter = converter; + this.tableSchema = tableSchema; + this.readSchema = readSchema(tableSchema, projectedSchema); + this.nameMapping = nameMapping; + this.caseSensitive = caseSensitive; + this.io = io; + this.encryption = encryption; + this.filters = filters; + this.limit = limit; + } + + @Override + protected DataIterator createDataIterator(IcebergSourceSplit split) { + RowDataFileScanTaskReader rowDataReader = + new RowDataFileScanTaskReader(tableSchema, readSchema, nameMapping, caseSensitive, filters); + return new LimitableDataIterator<>( + new ConverterFileScanTaskReader<>(rowDataReader, converter), + split.task(), + io, + encryption, + lazyLimiter()); + } + + private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { + Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); + return projectedSchema == null ? tableSchema : projectedSchema; + } + + /** Lazily create RecordLimiter to avoid the need to make it serializable */ + private RecordLimiter lazyLimiter() { + if (recordLimiter == null) { + this.recordLimiter = RecordLimiter.create(limit); + } + + return recordLimiter; + } + + private static class ConverterFileScanTaskReader implements FileScanTaskReader { + private final RowDataFileScanTaskReader rowDataReader; + private final RowDataConverter converter; + + ConverterFileScanTaskReader( + RowDataFileScanTaskReader rowDataReader, RowDataConverter converter) { + this.rowDataReader = rowDataReader; + this.converter = converter; + } + + @Override + public CloseableIterator open( + FileScanTask fileScanTask, InputFilesDecryptor inputFilesDecryptor) { + return CloseableIterator.transform( + rowDataReader.open(fileScanTask, inputFilesDecryptor), converter); + } + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java index 9c20494fdbcd..bcd72e25036b 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java @@ -85,7 +85,7 @@ public RecordsWithSplitIds> fetch() throws IOException { } else { // return an empty result, which will lead to split fetch to be idle. // SplitFetcherManager will then close idle fetcher. - return new RecordsBySplits(Collections.emptyMap(), Collections.emptySet()); + return new RecordsBySplits<>(Collections.emptyMap(), Collections.emptySet()); } } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java index 6ac92592b6aa..10e7d2037a30 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.source.reader; +import java.util.Locale; import org.apache.flink.annotation.Internal; /** @@ -73,6 +74,6 @@ public void record(T nextRecord) { @Override public String toString() { - return String.format("%s @ %d + %d", record, fileOffset, recordOffset); + return String.format(Locale.ROOT, "%s @ %d + %d", record, fileOffset, recordOffset); } } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java new file mode 100644 index 000000000000..98bb7e981840 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import java.io.Serializable; +import java.util.function.Function; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.table.data.RowData; + +/** + * Convert RowData to a different output type. + * + * @param output type + */ +public interface RowDataConverter + extends Function, ResultTypeQueryable, Serializable {} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java index d4b0f9e1977d..d90d1dc88c91 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.source.split; import java.io.IOException; +import java.util.Locale; import org.apache.flink.annotation.Internal; import org.apache.flink.core.io.SimpleVersionedSerializer; @@ -54,6 +55,7 @@ public IcebergSourceSplit deserialize(int version, byte[] serialized) throws IOE default: throw new IOException( String.format( + Locale.ROOT, "Failed to deserialize IcebergSourceSplit. " + "Encountered unsupported version: %d. Supported version are [1]", version)); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java index 482cfd110bde..b63547d433a4 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.flink.FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HADOOP; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; import java.util.Arrays; @@ -46,6 +47,7 @@ import org.apache.iceberg.Parameter; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Parameters; +import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.Namespace; @@ -54,6 +56,7 @@ import org.apache.iceberg.hadoop.HadoopCatalog; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -241,4 +244,93 @@ public void testHashDistributeMode() throws Exception { sql("DROP TABLE IF EXISTS %s.%s", FLINK_DATABASE, tableName); } } + + @TestTemplate + public void testRangeDistributionPartitionColumn() { + // Range partitioner currently only works with streaming writes (with checkpoints) + assumeThat(isStreamingJob).isTrue(); + + // Initialize a BoundedSource table to precisely emit those rows in only one checkpoint. + List> rowsPerCheckpoint = + IntStream.range(1, 6) + .mapToObj( + checkpointId -> { + List charRows = Lists.newArrayList(); + // emit 26x10 rows for each checkpoint cycle + for (int i = 0; i < 10; ++i) { + for (char c = 'a'; c <= 'z'; c++) { + charRows.add(Row.of(c - 'a', String.valueOf(c))); + } + } + return charRows; + }) + .collect(Collectors.toList()); + List flattenedRows = + rowsPerCheckpoint.stream().flatMap(List::stream).collect(Collectors.toList()); + + String dataId = BoundedTableFactory.registerDataSet(rowsPerCheckpoint); + sql( + "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" + + " WITH ('connector'='BoundedSource', 'data-id'='%s')", + SOURCE_TABLE, dataId); + + assertThat(sql("SELECT * FROM %s", SOURCE_TABLE)) + .as("Should have the expected rows in source table.") + .containsExactlyInAnyOrderElementsOf(flattenedRows); + + Map tableProps = + ImmutableMap.of( + "write.format.default", + FileFormat.PARQUET.name(), + TableProperties.WRITE_DISTRIBUTION_MODE, + DistributionMode.RANGE.modeName()); + + String tableName = "test_hash_distribution_mode"; + sql( + "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH %s", + tableName, toWithClause(tableProps)); + + try { + // Insert data set. + sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); + + assertThat(sql("SELECT * FROM %s", tableName)) + .as("Should have the expected rows in sink table.") + .containsExactlyInAnyOrderElementsOf(flattenedRows); + + Table table = catalog.loadTable(TableIdentifier.of(ICEBERG_NAMESPACE, tableName)); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(5); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + for (Snapshot snapshot : rangePartitionedCycles) { + List addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // range partition results in each partition only assigned to one writer task + // maybe less than 26 partitions as BoundedSource doesn't always precisely + // control the checkpoint boundary. + // It is hard to precisely control the test condition in SQL tests. + // Here only minimal safe assertions are applied to avoid flakiness. + // If there are no shuffling, the number of data files could be as high as + // 26 * 4 as the default parallelism is set to 4 for the mini cluster. + assertThat(addedDataFiles).hasSizeLessThanOrEqualTo(26); + } + } finally { + sql("DROP TABLE IF EXISTS %s.%s", FLINK_DATABASE, tableName); + } + } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ConstantsForTests.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ConstantsForTests.java new file mode 100644 index 000000000000..36e162d4f068 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ConstantsForTests.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +class ConstantsForTests { + public static final long EVENT_TIME = 10L; + static final long EVENT_TIME_2 = 11L; + static final String DUMMY_NAME = "dummy"; + + private ConstantsForTests() { + // Do not instantiate + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ManualSource.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ManualSource.java index 38bb9c393fa9..679b3ec508a2 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ManualSource.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ManualSource.java @@ -22,6 +22,7 @@ import java.util.Collections; import java.util.List; import java.util.concurrent.CompletableFuture; +import org.apache.flink.api.common.eventtime.Watermark; import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.Boundedness; @@ -156,23 +157,26 @@ public SimpleVersionedSerializer getEnumeratorCheckpointSeriali @Override public SourceReader createReader(SourceReaderContext sourceReaderContext) { - return new SourceReader() { + return new SourceReader<>() { @Override public void start() { // Do nothing } + @SuppressWarnings("unchecked") @Override public InputStatus pollNext(ReaderOutput output) { Tuple2 next = (Tuple2) QUEUES.get(index).poll(); if (next != null) { if (next.f0 == null) { - // No more input - return InputStatus.END_OF_INPUT; - } - - if (next.f1 == null) { + if (next.f1 == null) { + // No more input + return InputStatus.END_OF_INPUT; + } else { + output.emitWatermark(new Watermark(next.f1)); + } + } else if (next.f1 == null) { // No event time set output.collect(next.f0); } else { diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java new file mode 100644 index 000000000000..7a523035b7fb --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java @@ -0,0 +1,153 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Arrays; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricConfig; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.reporter.MetricReporter; +import org.apache.flink.metrics.reporter.MetricReporterFactory; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +public class MetricsReporterFactoryForTests implements MetricReporterFactory { + private static final TestMetricsReporter INSTANCE = new TestMetricsReporter(); + private static final Pattern FULL_METRIC_NAME = + Pattern.compile( + "\\.taskmanager\\.[^.]+\\.[^.]+\\.([^.]+)\\.\\d+\\." + + TableMaintenanceMetrics.GROUP_KEY + + "\\.([^.]+)\\.([^.]+)"); + + private static Map counters = Maps.newConcurrentMap(); + private static Map gauges = Maps.newConcurrentMap(); + private static Set monitoredMetricNames; + + public MetricsReporterFactoryForTests() { + monitoredMetricNames = + Arrays.stream(TableMaintenanceMetrics.class.getDeclaredFields()) + .map( + f -> { + try { + return f.get(null).toString(); + } catch (IllegalAccessException e) { + throw new RuntimeException(e); + } + }) + .collect(Collectors.toSet()); + } + + @Override + public MetricReporter createMetricReporter(Properties properties) { + return INSTANCE; + } + + public static void reset() { + counters = Maps.newConcurrentMap(); + gauges = Maps.newConcurrentMap(); + } + + public static Long counter(String name) { + return counterValues().get(name); + } + + public static Long gauge(String name) { + return gaugeValues().get(name); + } + + public static void assertGauges(Map expected) { + assertThat(filter(gaugeValues(), expected)).isEqualTo(filter(expected, expected)); + } + + public static void assertCounters(Map expected) { + assertThat(filter(counterValues(), expected)).isEqualTo(filter(expected, expected)); + } + + private static Map gaugeValues() { + return gauges.entrySet().stream() + .collect( + Collectors.toMap( + entry -> longName(entry.getKey()), entry -> (Long) entry.getValue().getValue())); + } + + private static Map counterValues() { + return counters.entrySet().stream() + .collect( + Collectors.toMap( + entry -> longName(entry.getKey()), entry -> entry.getValue().getCount())); + } + + private static Map filter(Map original, Map filter) { + return original.entrySet().stream() + .filter( + entry -> { + Long filterValue = filter.get(entry.getKey()); + return filterValue == null || filterValue != -1; + }) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + } + + private static String longName(String fullName) { + Matcher matcher = FULL_METRIC_NAME.matcher(fullName); + if (!matcher.matches()) { + throw new RuntimeException(String.format("Can't parse simplified metrics name %s", fullName)); + } + + return matcher.group(1) + "." + matcher.group(2) + "." + matcher.group(3); + } + + private static class TestMetricsReporter implements MetricReporter { + @Override + public void open(MetricConfig config) { + // do nothing + } + + @Override + public void close() { + // do nothing + } + + @Override + public void notifyOfAddedMetric(Metric metric, String metricName, MetricGroup group) { + if (monitoredMetricNames.contains(metricName)) { + if (metric instanceof Counter) { + counters.put(group.getMetricIdentifier(metricName), (Counter) metric); + } + + if (metric instanceof Gauge) { + gauges.put(group.getMetricIdentifier(metricName), (Gauge) metric); + } + } + } + + @Override + public void notifyOfRemovedMetric(Metric metric, String metricName, MetricGroup group) { + // do nothing + } + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java index 272e0b693fd3..225853086545 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java @@ -20,16 +20,24 @@ import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; +import java.io.File; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.MetricOptions; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.core.execution.SavepointFormatType; +import org.apache.flink.runtime.jobgraph.SavepointConfigOptions; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.iceberg.flink.FlinkCatalogFactory; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.awaitility.Awaitility; import org.junit.jupiter.api.extension.RegisterExtension; class OperatorTestBase { private static final int NUMBER_TASK_MANAGERS = 1; private static final int SLOTS_PER_TASK_MANAGER = 8; + private static final TriggerLockFactory.Lock MAINTENANCE_LOCK = new MemoryLock(); + private static final TriggerLockFactory.Lock RECOVERY_LOCK = new MemoryLock(); static final String TABLE_NAME = "test_table"; @@ -39,7 +47,7 @@ class OperatorTestBase { new MiniClusterResourceConfiguration.Builder() .setNumberTaskManagers(NUMBER_TASK_MANAGERS) .setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER) - .setConfiguration(new Configuration(DISABLE_CLASSLOADER_CHECK_CONFIG)) + .setConfiguration(config()) .build()); @RegisterExtension @@ -48,4 +56,101 @@ class OperatorTestBase { "catalog", ImmutableMap.of("type", "iceberg", FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hadoop"), "db"); + + private static Configuration config() { + Configuration config = new Configuration(DISABLE_CLASSLOADER_CHECK_CONFIG); + MetricOptions.forReporter(config, "test_reporter") + .set(MetricOptions.REPORTER_FACTORY_CLASS, MetricsReporterFactoryForTests.class.getName()); + return config; + } + + protected static TriggerLockFactory lockFactory() { + return new TriggerLockFactory() { + @Override + public void open() { + MAINTENANCE_LOCK.unlock(); + RECOVERY_LOCK.unlock(); + } + + @Override + public Lock createLock() { + return MAINTENANCE_LOCK; + } + + @Override + public Lock createRecoveryLock() { + return RECOVERY_LOCK; + } + + @Override + public void close() { + // do nothing + } + }; + } + + /** + * Close the {@link JobClient} and wait for the job closure. If the savepointDir is specified, it + * stops the job with a savepoint. + * + * @param jobClient the job to close + * @param savepointDir the savepointDir to store the last savepoint. If null then + * stop without a savepoint. + * @return configuration for restarting the job from the savepoint + */ + public static Configuration closeJobClient(JobClient jobClient, File savepointDir) { + Configuration conf = new Configuration(); + if (jobClient != null) { + if (savepointDir != null) { + // Stop with savepoint + jobClient.stopWithSavepoint(false, savepointDir.getPath(), SavepointFormatType.CANONICAL); + // Wait until the savepoint is created and the job has been stopped + Awaitility.await().until(() -> savepointDir.listFiles(File::isDirectory).length == 1); + conf.set( + SavepointConfigOptions.SAVEPOINT_PATH, + savepointDir.listFiles(File::isDirectory)[0].getAbsolutePath()); + } else { + jobClient.cancel(); + } + + // Wait until the job has been stopped + Awaitility.await().until(() -> jobClient.getJobStatus().get().isTerminalState()); + return conf; + } + + return null; + } + + /** + * Close the {@link JobClient} and wait for the job closure. + * + * @param jobClient the job to close + */ + public static void closeJobClient(JobClient jobClient) { + closeJobClient(jobClient, null); + } + + private static class MemoryLock implements TriggerLockFactory.Lock { + boolean locked = false; + + @Override + public boolean tryLock() { + if (locked) { + return false; + } else { + locked = true; + return true; + } + } + + @Override + public boolean isHeld() { + return locked; + } + + @Override + public void unlock() { + locked = false; + } + } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestJdbcLockFactory.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestJdbcLockFactory.java new file mode 100644 index 000000000000..051d09d92bad --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestJdbcLockFactory.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import static org.apache.iceberg.flink.maintenance.operator.JdbcLockFactory.INIT_LOCK_TABLES_PROPERTY; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.jdbc.JdbcCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.Test; + +class TestJdbcLockFactory extends TestLockFactoryBase { + @Override + TriggerLockFactory lockFactory() { + return lockFactory("tableName"); + } + + @Test + void testMultiTableLock() { + JdbcLockFactory other = lockFactory("tableName2"); + other.open((JdbcLockFactory) this.lockFactory); + TriggerLockFactory.Lock lock1 = lockFactory.createLock(); + TriggerLockFactory.Lock lock2 = other.createLock(); + assertThat(lock1.tryLock()).isTrue(); + assertThat(lock2.tryLock()).isTrue(); + } + + private JdbcLockFactory lockFactory(String tableName) { + Map properties = Maps.newHashMap(); + properties.put(JdbcCatalog.PROPERTY_PREFIX + "username", "user"); + properties.put(JdbcCatalog.PROPERTY_PREFIX + "password", "password"); + properties.put(INIT_LOCK_TABLES_PROPERTY, "true"); + + return new JdbcLockFactory( + "jdbc:sqlite:file::memory:?ic" + UUID.randomUUID().toString().replace("-", ""), + tableName, + properties); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockFactoryBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockFactoryBase.java new file mode 100644 index 000000000000..bf9e86f2534d --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockFactoryBase.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +abstract class TestLockFactoryBase { + protected TriggerLockFactory lockFactory; + + abstract TriggerLockFactory lockFactory(); + + @BeforeEach + void before() { + this.lockFactory = lockFactory(); + lockFactory.open(); + } + + @AfterEach + void after() throws IOException { + lockFactory.close(); + } + + @Test + void testTryLock() { + TriggerLockFactory.Lock lock1 = lockFactory.createLock(); + TriggerLockFactory.Lock lock2 = lockFactory.createLock(); + assertThat(lock1.tryLock()).isTrue(); + assertThat(lock1.tryLock()).isFalse(); + assertThat(lock2.tryLock()).isFalse(); + } + + @Test + void testUnLock() { + TriggerLockFactory.Lock lock = lockFactory.createLock(); + assertThat(lock.tryLock()).isTrue(); + + lock.unlock(); + assertThat(lock.tryLock()).isTrue(); + } + + @Test + void testNoConflictWithRecoveryLock() { + TriggerLockFactory.Lock lock1 = lockFactory.createLock(); + TriggerLockFactory.Lock lock2 = lockFactory.createRecoveryLock(); + assertThat(lock1.tryLock()).isTrue(); + assertThat(lock2.tryLock()).isTrue(); + } + + @Test + void testDoubleUnLock() { + TriggerLockFactory.Lock lock = lockFactory.createLock(); + assertThat(lock.tryLock()).isTrue(); + + lock.unlock(); + lock.unlock(); + assertThat(lock.tryLock()).isTrue(); + assertThat(lock.tryLock()).isFalse(); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java new file mode 100644 index 000000000000..cffcc4eb0471 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java @@ -0,0 +1,378 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import static org.apache.iceberg.flink.maintenance.operator.ConstantsForTests.DUMMY_NAME; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.FAILED_TASK_COUNTER; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.SUCCEEDED_TASK_COUNTER; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.util.Collection; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.api.connector.sink2.CommitterInitContext; +import org.apache.flink.api.connector.sink2.CommittingSinkWriter; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.api.connector.sink2.SupportsCommitter; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; +import org.apache.flink.streaming.api.connector.sink2.SupportsPostCommitTopology; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.util.Collector; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.io.TempDir; + +@Timeout(value = 10) +class TestLockRemover extends OperatorTestBase { + private static final String[] TASKS = new String[] {"task0", "task1"}; + private static final TriggerLockFactory.Lock LOCK = new TestingLock(); + private static final TriggerLockFactory.Lock RECOVERY_LOCK = new TestingLock(); + + @TempDir private File checkpointDir; + + @BeforeEach + void before() { + MetricsReporterFactoryForTests.reset(); + } + + @Test + void testProcess() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource source = new ManualSource<>(env, TypeInformation.of(TaskResult.class)); + source + .dataStream() + .transform( + DUMMY_NAME, + TypeInformation.of(Void.class), + new LockRemover(new TestingLockFactory(), Lists.newArrayList(TASKS))) + .setParallelism(1); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + LOCK.tryLock(); + assertThat(LOCK.isHeld()).isTrue(); + + // Start a successful trigger for task1 and assert the return value is correct + processAndCheck(source, new TaskResult(0, 0L, true, Lists.newArrayList())); + + // Assert that the lock is removed + assertThat(LOCK.isHeld()).isFalse(); + } finally { + closeJobClient(jobClient); + } + } + + @Test + void testInSink() throws Exception { + String sinkName = "TestSink"; + Configuration config = new Configuration(); + config.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem"); + config.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, "file://" + checkpointDir.getPath()); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(config); + env.enableCheckpointing(10); + ManualSource source = new ManualSource<>(env, TypeInformation.of(TaskResult.class)); + source.dataStream().global().sinkTo(new SinkTest()).name(sinkName).setParallelism(1); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + LOCK.tryLock(); + assertThat(LOCK.isHeld()).isTrue(); + + // Start a successful trigger for task1 and assert the return value is correct + processAndCheck(source, new TaskResult(0, 0L, true, Lists.newArrayList()), sinkName + ": "); + + // Assert that the lock is removed + assertThat(LOCK.isHeld()).isFalse(); + } finally { + closeJobClient(jobClient); + } + } + + @Test + void testMetrics() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource source = new ManualSource<>(env, TypeInformation.of(TaskResult.class)); + source + .dataStream() + .transform( + DUMMY_NAME, + TypeInformation.of(Void.class), + new LockRemover(new TestingLockFactory(), Lists.newArrayList(TASKS))) + .setParallelism(1); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + // Start the 2 successful and one failed result trigger for task1, and 3 successful for task2 + processAndCheck(source, new TaskResult(0, 0L, true, Lists.newArrayList())); + processAndCheck(source, new TaskResult(1, 1L, true, Lists.newArrayList())); + processAndCheck(source, new TaskResult(1, 2L, true, Lists.newArrayList())); + processAndCheck(source, new TaskResult(0, 3L, false, Lists.newArrayList())); + processAndCheck(source, new TaskResult(0, 4L, true, Lists.newArrayList())); + processAndCheck(source, new TaskResult(1, 5L, true, Lists.newArrayList())); + + Awaitility.await() + .until( + () -> + MetricsReporterFactoryForTests.counter( + DUMMY_NAME + "." + TASKS[1] + "." + SUCCEEDED_TASK_COUNTER) + .equals(3L)); + + // Final check all the counters + MetricsReporterFactoryForTests.assertCounters( + new ImmutableMap.Builder() + .put(DUMMY_NAME + "." + TASKS[0] + "." + SUCCEEDED_TASK_COUNTER, 2L) + .put(DUMMY_NAME + "." + TASKS[0] + "." + FAILED_TASK_COUNTER, 1L) + .put(DUMMY_NAME + "." + TASKS[1] + "." + SUCCEEDED_TASK_COUNTER, 3L) + .put(DUMMY_NAME + "." + TASKS[1] + "." + FAILED_TASK_COUNTER, 0L) + .build()); + } finally { + closeJobClient(jobClient); + } + } + + /** + * The test checks if the recovery watermark is only removed if the watermark has arrived from + * both upstream sources. + * + * @throws Exception if any + */ + @Test + void testRecovery() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource source1 = + new ManualSource<>(env, TypeInformation.of(TaskResult.class)); + ManualSource source2 = + new ManualSource<>(env, TypeInformation.of(TaskResult.class)); + source1 + .dataStream() + .union(source2.dataStream()) + .transform( + DUMMY_NAME, + TypeInformation.of(Void.class), + new LockRemover(new TestingLockFactory(), Lists.newArrayList(TASKS[0]))) + .setParallelism(1); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + RECOVERY_LOCK.tryLock(); + assertThat(RECOVERY_LOCK.isHeld()).isTrue(); + + processAndCheck(source1, new TaskResult(0, 0L, true, Lists.newArrayList())); + + source1.sendRecord(new TaskResult(0, 1L, true, Lists.newArrayList())); + // we receive the second result - this will not happen in real use cases, but with this we can + // be sure that the previous watermark is processed + Awaitility.await() + .until( + () -> + MetricsReporterFactoryForTests.counter( + DUMMY_NAME + "." + TASKS[0] + "." + SUCCEEDED_TASK_COUNTER) + .equals(2L)); + + // We did not remove the recovery lock, as no watermark received from the other source + assertThat(RECOVERY_LOCK.isHeld()).isTrue(); + + // Recovery arrives + source1.sendWatermark(10L); + source2.sendWatermark(10L); + + Awaitility.await().until(() -> !RECOVERY_LOCK.isHeld()); + } finally { + closeJobClient(jobClient); + } + } + + private void processAndCheck(ManualSource source, TaskResult input) { + processAndCheck(source, input, null); + } + + private void processAndCheck( + ManualSource source, TaskResult input, String counterPrefix) { + source.sendRecord(input); + source.sendWatermark(input.startEpoch()); + + String counterName = + (counterPrefix != null ? counterPrefix : "") + .concat( + input.success() + ? DUMMY_NAME + "." + TASKS[input.taskIndex()] + "." + SUCCEEDED_TASK_COUNTER + : DUMMY_NAME + "." + TASKS[input.taskIndex()] + "." + FAILED_TASK_COUNTER); + Long counterValue = MetricsReporterFactoryForTests.counter(counterName); + Long expected = counterValue != null ? counterValue + 1 : 1L; + + Awaitility.await() + .until(() -> expected.equals(MetricsReporterFactoryForTests.counter(counterName))); + } + + private static class TestingLockFactory implements TriggerLockFactory { + @Override + public void open() { + // Do nothing + } + + @Override + public Lock createLock() { + return LOCK; + } + + @Override + public Lock createRecoveryLock() { + return RECOVERY_LOCK; + } + + @Override + public void close() { + // Do nothing + } + } + + private static class TestingLock implements TriggerLockFactory.Lock { + private boolean locked = false; + + @Override + public boolean tryLock() { + if (isHeld()) { + return false; + } else { + locked = true; + return true; + } + } + + @Override + public boolean isHeld() { + return locked; + } + + @Override + public void unlock() { + locked = false; + } + } + + private static class SinkTest + implements Sink, + SupportsCommitter, + SupportsPostCommitTopology { + @Override + public SinkWriter createWriter(InitContext initContext) { + return new CommittingSinkWriter() { + private final Collection received = Lists.newArrayList(); + + @Override + public Collection prepareCommit() { + Collection result = Lists.newArrayList(received); + received.clear(); + return result; + } + + @Override + public void write(TaskResult taskResult, Context context) { + received.add(taskResult); + } + + @Override + public void flush(boolean b) { + // noop + } + + @Override + public void close() { + // noop + } + }; + } + + @Override + public Committer createCommitter(CommitterInitContext committerInitContext) { + return new Committer<>() { + @Override + public void commit(Collection> collection) { + // noop + } + + @Override + public void close() { + // noop + } + }; + } + + @Override + public SimpleVersionedSerializer getCommittableSerializer() { + return new SimpleVersionedSerializer<>() { + @Override + public int getVersion() { + return 0; + } + + @Override + public byte[] serialize(TaskResult taskResult) { + return new byte[0]; + } + + @Override + public TaskResult deserialize(int i, byte[] bytes) { + return null; + } + }; + } + + @Override + public void addPostCommitTopology(DataStream> committables) { + committables + .flatMap( + new FlatMapFunction, TaskResult>() { + @Override + public void flatMap( + CommittableMessage taskResultCommittableMessage, + Collector collector) { + if (taskResultCommittableMessage instanceof CommittableWithLineage) { + collector.collect( + ((CommittableWithLineage) taskResultCommittableMessage) + .getCommittable()); + } + } + }) + .transform( + DUMMY_NAME, + TypeInformation.of(Void.class), + new LockRemover(new TestingLockFactory(), Lists.newArrayList(TASKS[0]))); + } + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java index 876d64214560..3aee05322561 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.flink.maintenance.operator; -import static org.apache.iceberg.flink.maintenance.operator.FlinkStreamingTestUtils.closeJobClient; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -35,6 +34,7 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.hadoop.fs.Path; +import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; @@ -161,7 +161,12 @@ void testSource() throws Exception { } // The first non-empty event should contain the expected value - return newEvent.equals(new TableChange(1, 0, size, 0L, 1)); + return newEvent.equals( + TableChange.builder() + .dataFileCount(1) + .dataFileSizeInBytes(size) + .commitCount(1) + .build()); }); } finally { closeJobClient(jobClient); @@ -297,17 +302,17 @@ void testMaxReadBack() { new MonitorSource.TableChangeIterator(tableLoader, null, 1); // For a single maxReadBack we only get a single change - assertThat(iterator.next().commitNum()).isEqualTo(1); + assertThat(iterator.next().commitCount()).isEqualTo(1); iterator = new MonitorSource.TableChangeIterator(tableLoader, null, 2); // Expecting 2 commits/snapshots for maxReadBack=2 - assertThat(iterator.next().commitNum()).isEqualTo(2); + assertThat(iterator.next().commitCount()).isEqualTo(2); iterator = new MonitorSource.TableChangeIterator(tableLoader, null, Long.MAX_VALUE); // For maxReadBack Long.MAX_VALUE we get every change - assertThat(iterator.next().commitNum()).isEqualTo(3); + assertThat(iterator.next().commitCount()).isEqualTo(3); } @Test @@ -322,7 +327,7 @@ void testSkipReplace() { new MonitorSource.TableChangeIterator(tableLoader, null, Long.MAX_VALUE); // Read the current snapshot - assertThat(iterator.next().commitNum()).isEqualTo(1); + assertThat(iterator.next().commitCount()).isEqualTo(1); // Create a DataOperations.REPLACE snapshot Table table = tableLoader.loadTable(); @@ -348,15 +353,19 @@ private static TableChange tableChangeWithLastSnapshot(Table table, TableChange List deleteFiles = Lists.newArrayList(table.currentSnapshot().addedDeleteFiles(table.io()).iterator()); - long dataSize = dataFiles.stream().mapToLong(d -> d.fileSizeInBytes()).sum(); - long deleteSize = deleteFiles.stream().mapToLong(d -> d.fileSizeInBytes()).sum(); - boolean hasDelete = table.currentSnapshot().addedDeleteFiles(table.io()).iterator().hasNext(); - - return new TableChange( - previous.dataFileNum() + dataFiles.size(), - previous.deleteFileNum() + deleteFiles.size(), - previous.dataFileSize() + dataSize, - previous.deleteFileSize() + deleteSize, - previous.commitNum() + 1); + long dataSize = dataFiles.stream().mapToLong(ContentFile::fileSizeInBytes).sum(); + long deleteRecordCount = deleteFiles.stream().mapToLong(DeleteFile::recordCount).sum(); + + TableChange newChange = previous.copy(); + newChange.merge( + TableChange.builder() + .dataFileCount(dataFiles.size()) + .dataFileSizeInBytes(dataSize) + // Currently we only test with equality deletes + .eqDeleteFileCount(deleteFiles.size()) + .eqDeleteRecordCount(deleteRecordCount) + .commitCount(1) + .build()); + return newChange; } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java new file mode 100644 index 000000000000..fba4a12d9c6b --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java @@ -0,0 +1,686 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import static org.apache.iceberg.flink.maintenance.operator.ConstantsForTests.DUMMY_NAME; +import static org.apache.iceberg.flink.maintenance.operator.ConstantsForTests.EVENT_TIME; +import static org.apache.iceberg.flink.maintenance.operator.ConstantsForTests.EVENT_TIME_2; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.CONCURRENT_RUN_THROTTLED; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.GROUP_VALUE_DEFAULT; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.NOTHING_TO_TRIGGER; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.RATE_LIMITER_TRIGGERED; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.TRIGGERED; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.time.Duration; +import java.util.Iterator; +import java.util.List; +import java.util.stream.Stream; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.operators.KeyedProcessOperator; +import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +class TestTriggerManager extends OperatorTestBase { + private static final long DELAY = 10L; + private static final String NAME_1 = "name1"; + private static final String NAME_2 = "name2"; + private long processingTime = 0L; + private TriggerLockFactory lockFactory; + private TriggerLockFactory.Lock lock; + private TriggerLockFactory.Lock recoveringLock; + + @BeforeEach + void before() { + sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); + this.lockFactory = lockFactory(); + lockFactory.open(); + this.lock = lockFactory.createLock(); + this.recoveringLock = lockFactory.createRecoveryLock(); + lock.unlock(); + recoveringLock.unlock(); + MetricsReporterFactoryForTests.reset(); + } + + @AfterEach + void after() throws IOException { + lockFactory.close(); + } + + @Test + void testCommitCount() throws Exception { + TriggerManager manager = + manager(sql.tableLoader(TABLE_NAME), new TriggerEvaluator.Builder().commitCount(3).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(1).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(3).build(), 2); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(10).build(), 3); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(1).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(1).build(), 3); + + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(1).build(), 4); + } + } + + @Test + void testDataFileCount() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), new TriggerEvaluator.Builder().dataFileCount(3).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(1).build(), 0); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(3).build(), 2); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(5).build(), 3); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(1).build(), 3); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(2).build(), 4); + } + } + + @Test + void testDataFileSizeInBytes() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().dataFileSizeInBytes(3).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSizeInBytes(1L).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSizeInBytes(2L).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSizeInBytes(5L).build(), 2); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSizeInBytes(1L).build(), 2); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSizeInBytes(2L).build(), 3); + } + } + + @Test + void testPosDeleteFileCount() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().posDeleteFileCount(3).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(1).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(3).build(), 2); + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(10).build(), 3); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(1).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(1).build(), 3); + + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(1).build(), 4); + } + } + + @Test + void testPosDeleteRecordCount() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().posDeleteRecordCount(3).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult( + testHarness, TableChange.builder().posDeleteRecordCount(1L).build(), 0); + addEventAndCheckResult( + testHarness, TableChange.builder().posDeleteRecordCount(2L).build(), 1); + addEventAndCheckResult( + testHarness, TableChange.builder().posDeleteRecordCount(5L).build(), 2); + + // No trigger in this case + addEventAndCheckResult( + testHarness, TableChange.builder().posDeleteRecordCount(1L).build(), 2); + + addEventAndCheckResult( + testHarness, TableChange.builder().posDeleteRecordCount(2L).build(), 3); + } + } + + @Test + void testEqDeleteFileCount() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().eqDeleteFileCount(3).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(1).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(3).build(), 2); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(10).build(), 3); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(1).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(1).build(), 3); + + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(1).build(), 4); + } + } + + @Test + void testEqDeleteRecordCount() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().eqDeleteRecordCount(3).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteRecordCount(1L).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteRecordCount(2L).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteRecordCount(5L).build(), 2); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteRecordCount(1L).build(), 2); + + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteRecordCount(2L).build(), 3); + } + } + + @Test + void testTimeout() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().timeout(Duration.ofSeconds(1)).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + TableChange event = TableChange.builder().dataFileCount(1).commitCount(1).build(); + + // Wait for some time + testHarness.processElement(event, EVENT_TIME); + assertThat(testHarness.extractOutputValues()).isEmpty(); + + // Wait for the timeout to expire + long newTime = EVENT_TIME + Duration.ofSeconds(1).toMillis(); + testHarness.setProcessingTime(newTime); + testHarness.processElement(event, newTime); + assertThat(testHarness.extractOutputValues()).hasSize(1); + + // Remove the lock to allow the next trigger + lock.unlock(); + + // Send a new event + testHarness.setProcessingTime(newTime + 1); + testHarness.processElement(event, newTime); + + // No trigger yet + assertThat(testHarness.extractOutputValues()).hasSize(1); + + // Send a new event + newTime += Duration.ofSeconds(1).toMillis(); + testHarness.setProcessingTime(newTime); + testHarness.processElement(event, newTime); + + // New trigger should arrive + assertThat(testHarness.extractOutputValues()).hasSize(2); + } + } + + @Test + void testStateRestore() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TriggerManager manager = manager(tableLoader); + OperatorSubtaskState state; + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + testHarness.processElement( + TableChange.builder().dataFileCount(1).commitCount(1).build(), EVENT_TIME); + + assertThat(testHarness.extractOutputValues()).isEmpty(); + + state = testHarness.snapshot(1, EVENT_TIME); + } + + // Restore the state, write some more data, create a checkpoint, check the data which is written + manager = manager(tableLoader); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.initializeState(state); + testHarness.open(); + + // Arrives the first real change which triggers the recovery process + testHarness.processElement(TableChange.builder().commitCount(1).build(), EVENT_TIME_2); + assertTriggers( + testHarness.extractOutputValues(), + Lists.newArrayList(Trigger.recovery(testHarness.getProcessingTime()))); + + // Remove the lock to allow the next trigger + recoveringLock.unlock(); + testHarness.setProcessingTime(EVENT_TIME_2); + // At this point the output contains the recovery trigger and the real trigger + assertThat(testHarness.extractOutputValues()).hasSize(2); + } + } + + @Test + void testMinFireDelay() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TriggerManager manager = manager(tableLoader, DELAY, 1); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(2).build(), 1); + long currentTime = testHarness.getProcessingTime(); + + // No new fire yet + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(2).build(), 1); + + // Check that the trigger fired after the delay + testHarness.setProcessingTime(currentTime + DELAY); + assertThat(testHarness.extractOutputValues()).hasSize(2); + } + } + + @Test + void testLockCheckDelay() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TriggerManager manager = manager(tableLoader, 1, DELAY); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(2).build(), 1); + + // Create a lock to prevent execution, and check that there is no result + assertThat(lock.tryLock()).isTrue(); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(2).build(), 1); + long currentTime = testHarness.getProcessingTime(); + + // Remove the lock, and still no trigger + lock.unlock(); + assertThat(testHarness.extractOutputValues()).hasSize(1); + + // Check that the trigger fired after the delay + testHarness.setProcessingTime(currentTime + DELAY); + assertThat(testHarness.extractOutputValues()).hasSize(2); + } + } + + /** + * Simulating recovery scenarios where there is a leftover table lock, and ongoing maintenance + * task. + * + * @param locked if a lock exists on the table on job recovery + * @param runningTask is running and continues to run after job recovery + */ + @ParameterizedTest + @MethodSource("parametersForTestRecovery") + void testRecovery(boolean locked, boolean runningTask) throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TriggerManager manager = manager(tableLoader); + OperatorSubtaskState state; + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + state = testHarness.snapshot(1, EVENT_TIME); + } + + if (locked) { + assertThat(lock.tryLock()).isTrue(); + } + + manager = manager(tableLoader); + List expected = Lists.newArrayListWithExpectedSize(3); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.initializeState(state); + testHarness.open(); + + ++processingTime; + expected.add(Trigger.recovery(processingTime)); + testHarness.setProcessingTime(processingTime); + testHarness.processElement(TableChange.builder().commitCount(2).build(), processingTime); + assertTriggers(testHarness.extractOutputValues(), expected); + + // Nothing happens until the recovery is finished + ++processingTime; + testHarness.setProcessingTime(processingTime); + assertTriggers(testHarness.extractOutputValues(), expected); + + if (runningTask) { + // Simulate the action of the recovered maintenance task lock removal when it finishes + lock.unlock(); + } + + // Still no results as the recovery is ongoing + ++processingTime; + testHarness.setProcessingTime(processingTime); + testHarness.processElement(TableChange.builder().commitCount(2).build(), processingTime); + assertTriggers(testHarness.extractOutputValues(), expected); + + // Simulate the action of removing lock and recoveryLock by downstream lock cleaner when it + // received recovery trigger + lock.unlock(); + recoveringLock.unlock(); + + // Emit only a single trigger + ++processingTime; + testHarness.setProcessingTime(processingTime); + // Releasing lock will create a new snapshot, and we receive this in the trigger + expected.add( + Trigger.create( + processingTime, + (SerializableTable) SerializableTable.copyOf(tableLoader.loadTable()), + 0)); + assertTriggers(testHarness.extractOutputValues(), expected); + } + } + + @Test + void testTriggerMetrics() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource source = + new ManualSource<>(env, TypeInformation.of(TableChange.class)); + CollectingSink sink = new CollectingSink<>(); + + TriggerManager manager = + new TriggerManager( + tableLoader, + lockFactory, + Lists.newArrayList(NAME_1, NAME_2), + Lists.newArrayList( + new TriggerEvaluator.Builder().commitCount(2).build(), + new TriggerEvaluator.Builder().commitCount(4).build()), + 1L, + 1L); + source + .dataStream() + .keyBy(unused -> true) + .process(manager) + .name(DUMMY_NAME) + .forceNonParallel() + .sinkTo(sink); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + // This one doesn't trigger - tests NOTHING_TO_TRIGGER + source.sendRecord(TableChange.builder().commitCount(1).build()); + + Awaitility.await() + .until( + () -> { + Long notingCounter = + MetricsReporterFactoryForTests.counter( + DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + NOTHING_TO_TRIGGER); + return notingCounter != null && notingCounter.equals(1L); + }); + + // Trigger one of the tasks - tests TRIGGERED + source.sendRecord(TableChange.builder().commitCount(1).build()); + // Wait until we receive the trigger + assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); + assertThat( + MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED)) + .isEqualTo(1L); + lock.unlock(); + + // Trigger both of the tasks - tests TRIGGERED + source.sendRecord(TableChange.builder().commitCount(2).build()); + // Wait until we receive the trigger + assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); + lock.unlock(); + assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); + lock.unlock(); + assertThat( + MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED)) + .isEqualTo(2L); + assertThat( + MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + NAME_2 + "." + TRIGGERED)) + .isEqualTo(1L); + + // Final check all the counters + MetricsReporterFactoryForTests.assertCounters( + new ImmutableMap.Builder() + .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + RATE_LIMITER_TRIGGERED, -1L) + .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + CONCURRENT_RUN_THROTTLED, -1L) + .put(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED, 2L) + .put(DUMMY_NAME + "." + NAME_2 + "." + TRIGGERED, 1L) + .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + NOTHING_TO_TRIGGER, 1L) + .build()); + } finally { + closeJobClient(jobClient); + } + } + + @Test + void testRateLimiterMetrics() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource source = + new ManualSource<>(env, TypeInformation.of(TableChange.class)); + CollectingSink sink = new CollectingSink<>(); + + // High delay, so only triggered once + TriggerManager manager = manager(tableLoader, 1_000_000L, 1L); + source + .dataStream() + .keyBy(unused -> true) + .process(manager) + .name(DUMMY_NAME) + .forceNonParallel() + .sinkTo(sink); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + // Start the first trigger + source.sendRecord(TableChange.builder().commitCount(2).build()); + assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); + + // Remove the lock to allow the next trigger + lock.unlock(); + + // The second trigger will be blocked + source.sendRecord(TableChange.builder().commitCount(2).build()); + Awaitility.await() + .until( + () -> + MetricsReporterFactoryForTests.counter( + DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + RATE_LIMITER_TRIGGERED) + .equals(1L)); + + // Final check all the counters + assertCounters(1L, 0L); + } finally { + closeJobClient(jobClient); + } + } + + @Test + void testConcurrentRunMetrics() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource source = + new ManualSource<>(env, TypeInformation.of(TableChange.class)); + CollectingSink sink = new CollectingSink<>(); + + // High delay, so only triggered once + TriggerManager manager = manager(tableLoader, 1L, 1_000_000L); + source + .dataStream() + .keyBy(unused -> true) + .process(manager) + .name(DUMMY_NAME) + .forceNonParallel() + .sinkTo(sink); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + // Start the first trigger - notice that we do not remove the lock after the trigger + source.sendRecord(TableChange.builder().commitCount(2).build()); + assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); + + // The second trigger will be blocked by the lock + source.sendRecord(TableChange.builder().commitCount(2).build()); + Awaitility.await() + .until( + () -> + MetricsReporterFactoryForTests.counter( + DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + CONCURRENT_RUN_THROTTLED) + .equals(1L)); + + // Final check all the counters + assertCounters(0L, 1L); + } finally { + closeJobClient(jobClient); + } + } + + private static Stream parametersForTestRecovery() { + return Stream.of( + Arguments.of(true, false), + Arguments.of(true, false), + Arguments.of(false, true), + Arguments.of(false, false)); + } + + private void assertCounters(long rateLimiterTrigger, long concurrentRunTrigger) { + MetricsReporterFactoryForTests.assertCounters( + new ImmutableMap.Builder() + .put( + DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + RATE_LIMITER_TRIGGERED, + rateLimiterTrigger) + .put( + DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + CONCURRENT_RUN_THROTTLED, + concurrentRunTrigger) + .put(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED, 1L) + .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + NOTHING_TO_TRIGGER, 0L) + .build()); + } + + private KeyedOneInputStreamOperatorTestHarness harness( + TriggerManager manager) throws Exception { + return new KeyedOneInputStreamOperatorTestHarness<>( + new KeyedProcessOperator<>(manager), value -> true, Types.BOOLEAN); + } + + private void addEventAndCheckResult( + OneInputStreamOperatorTestHarness testHarness, + TableChange event, + int expectedSize) + throws Exception { + ++processingTime; + testHarness.setProcessingTime(processingTime); + testHarness.processElement(event, processingTime); + assertThat(testHarness.extractOutputValues()).hasSize(expectedSize); + // Remove the lock to allow the next trigger + lock.unlock(); + } + + private TriggerManager manager(TableLoader tableLoader, TriggerEvaluator evaluator) { + return new TriggerManager( + tableLoader, lockFactory, Lists.newArrayList(NAME_1), Lists.newArrayList(evaluator), 1, 1); + } + + private TriggerManager manager( + TableLoader tableLoader, long minFireDelayMs, long lockCheckDelayMs) { + return new TriggerManager( + tableLoader, + lockFactory, + Lists.newArrayList(NAME_1), + Lists.newArrayList(new TriggerEvaluator.Builder().commitCount(2).build()), + minFireDelayMs, + lockCheckDelayMs); + } + + private TriggerManager manager(TableLoader tableLoader) { + return manager(tableLoader, new TriggerEvaluator.Builder().commitCount(2).build()); + } + + private static void assertTriggers(List expected, List actual) { + assertThat(actual).hasSize(expected.size()); + for (int i = 0; i < expected.size(); ++i) { + Trigger expectedTrigger = expected.get(i); + Trigger actualTrigger = actual.get(i); + assertThat(actualTrigger.timestamp()).isEqualTo(expectedTrigger.timestamp()); + assertThat(actualTrigger.taskId()).isEqualTo(expectedTrigger.taskId()); + assertThat(actualTrigger.isRecovery()).isEqualTo(expectedTrigger.isRecovery()); + if (expectedTrigger.table() == null) { + assertThat(actualTrigger.table()).isNull(); + } else { + Iterator expectedSnapshots = expectedTrigger.table().snapshots().iterator(); + Iterator actualSnapshots = actualTrigger.table().snapshots().iterator(); + while (expectedSnapshots.hasNext()) { + assertThat(actualSnapshots.hasNext()).isTrue(); + assertThat(expectedSnapshots.next().snapshotId()) + .isEqualTo(actualSnapshots.next().snapshotId()); + } + } + } + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/SinkTestUtil.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/SinkTestUtil.java new file mode 100644 index 000000000000..1cf55bcdc817 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/SinkTestUtil.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Collection; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableSummary; +import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; +import org.apache.flink.streaming.runtime.streamrecord.StreamElement; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + +class SinkTestUtil { + + private SinkTestUtil() {} + + @SuppressWarnings("unchecked") + static List transformsToStreamElement(Collection elements) { + return elements.stream() + .map( + element -> { + if (element instanceof StreamRecord) { + return new StreamRecord<>( + ((StreamRecord>) element).getValue()); + } + return (StreamElement) element; + }) + .collect(Collectors.toList()); + } + + static CommittableSummary extractAndAssertCommittableSummary(StreamElement element) { + final Object value = element.asRecord().getValue(); + assertThat(value).isInstanceOf(CommittableSummary.class); + return (CommittableSummary) value; + } + + static CommittableWithLineage extractAndAssertCommittableWithLineage( + StreamElement element) { + final Object value = element.asRecord().getValue(); + assertThat(value).isInstanceOf(CommittableWithLineage.class); + return (CommittableWithLineage) value; + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java index 8faae1b05a4e..3299e7a97776 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java @@ -40,7 +40,6 @@ import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.io.BaseTaskWriter; import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -210,8 +209,10 @@ public void testCompressionOrc() throws Exception { .containsEntry(TableProperties.ORC_COMPRESSION_STRATEGY, "speed"); } - private static OneInputStreamOperatorTestHarness createIcebergStreamWriter( - Table icebergTable, TableSchema flinkSchema, Map override) throws Exception { + private static OneInputStreamOperatorTestHarness + createIcebergStreamWriter( + Table icebergTable, TableSchema flinkSchema, Map override) + throws Exception { RowType flinkRowType = FlinkSink.toFlinkRowType(icebergTable.schema(), flinkSchema); FlinkWriteConf flinkWriteConfig = new FlinkWriteConf( @@ -219,7 +220,7 @@ private static OneInputStreamOperatorTestHarness createIce IcebergStreamWriter streamWriter = FlinkSink.createStreamWriter(() -> icebergTable, flinkWriteConfig, flinkRowType, null); - OneInputStreamOperatorTestHarness harness = + OneInputStreamOperatorTestHarness harness = new OneInputStreamOperatorTestHarness<>(streamWriter, 1, 1, 0); harness.setup(); @@ -230,7 +231,7 @@ private static OneInputStreamOperatorTestHarness createIce private static Map appenderProperties( Table table, TableSchema schema, Map override) throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter(table, schema, override)) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java index 75e397d3f203..df8c3c79d3e3 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java @@ -20,28 +20,37 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.IOException; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; import org.apache.iceberg.DistributionMode; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Parameter; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; import org.apache.iceberg.TableProperties; import org.apache.iceberg.flink.FlinkWriteOptions; import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.sink.shuffle.StatisticsType; +import org.apache.iceberg.flink.source.BoundedTestSource; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Types; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -177,4 +186,309 @@ public void testOverrideWriteConfigWithUnknownDistributionMode() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid distribution mode: UNRECOGNIZED"); } + + @TestTemplate + public void testRangeDistributionWithoutSortOrderUnpartitioned() throws Exception { + assumeThat(partitioned).isFalse(); + + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + + int numOfCheckpoints = 6; + DataStream dataStream = + env.addSource( + createRangeDistributionBoundedSource(createCharRows(numOfCheckpoints, 10)), + ROW_TYPE_INFO); + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism); + + // Range distribution requires either sort order or partition spec defined + assertThatThrownBy(builder::append) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Invalid write distribution mode: range. Need to define sort order or partition spec."); + } + + @TestTemplate + public void testRangeDistributionWithoutSortOrderPartitioned() throws Exception { + assumeThat(partitioned).isTrue(); + + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + + int numOfCheckpoints = 6; + DataStream dataStream = + env.addSource( + createRangeDistributionBoundedSource(createCharRows(numOfCheckpoints, 10)), + ROW_TYPE_INFO); + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism); + + // sort based on partition columns + builder.append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(numOfCheckpoints); + } + + @TestTemplate + public void testRangeDistributionWithSortOrder() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + table.replaceSortOrder().asc("data").commit(); + + int numOfCheckpoints = 6; + DataStream dataStream = + env.addSource( + createRangeDistributionBoundedSource(createCharRows(numOfCheckpoints, 10)), + ROW_TYPE_INFO); + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .rangeDistributionStatisticsType(StatisticsType.Map) + .append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(numOfCheckpoints); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + if (partitioned) { + for (Snapshot snapshot : rangePartitionedCycles) { + List addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // up to 26 partitions + assertThat(addedDataFiles).hasSizeLessThanOrEqualTo(26); + } + } else { + for (Snapshot snapshot : rangePartitionedCycles) { + List addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // each writer task should only write one file for non-partition sort column + assertThat(addedDataFiles).hasSize(parallelism); + // verify there is no overlap in min-max stats range + if (parallelism == 2) { + assertIdColumnStatsNoRangeOverlap(addedDataFiles.get(0), addedDataFiles.get(1)); + } + } + } + } + + @TestTemplate + public void testRangeDistributionSketchWithSortOrder() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + table.replaceSortOrder().asc("id").commit(); + + int numOfCheckpoints = 6; + DataStream dataStream = + env.addSource( + createRangeDistributionBoundedSource(createIntRows(numOfCheckpoints, 1_000)), + ROW_TYPE_INFO); + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .rangeDistributionStatisticsType(StatisticsType.Sketch) + .append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(numOfCheckpoints); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + // since the input has a single value for the data column, + // it is always the same partition. Hence there is no difference + // for partitioned or not + for (Snapshot snapshot : rangePartitionedCycles) { + List addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // each writer task should only write one file for non-partition sort column + assertThat(addedDataFiles).hasSize(parallelism); + // verify there is no overlap in min-max stats range + if (parallelism == 2) { + assertIdColumnStatsNoRangeOverlap(addedDataFiles.get(0), addedDataFiles.get(1)); + } + } + } + + /** Test migration from Map stats to Sketch stats */ + @TestTemplate + public void testRangeDistributionStatisticsMigration() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + table.replaceSortOrder().asc("id").commit(); + + int numOfCheckpoints = 4; + List> rowsPerCheckpoint = Lists.newArrayListWithCapacity(numOfCheckpoints); + for (int checkpointId = 0; checkpointId < numOfCheckpoints; ++checkpointId) { + // checkpointId 2 would emit 11_000 records which is larger than + // the OPERATOR_SKETCH_SWITCH_THRESHOLD of 10_000. + // This should trigger the stats migration. + int maxId = checkpointId < 1 ? 1_000 : 11_000; + List rows = Lists.newArrayListWithCapacity(maxId); + for (int j = 0; j < maxId; ++j) { + // fixed value "a" for the data (possible partition column) + rows.add(Row.of(j, "a")); + } + + rowsPerCheckpoint.add(rows); + } + + DataStream dataStream = + env.addSource(createRangeDistributionBoundedSource(rowsPerCheckpoint), ROW_TYPE_INFO); + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .rangeDistributionStatisticsType(StatisticsType.Auto) + .append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(numOfCheckpoints); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + // since the input has a single value for the data column, + // it is always the same partition. Hence there is no difference + // for partitioned or not + for (Snapshot snapshot : rangePartitionedCycles) { + List addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // each writer task should only write one file for non-partition sort column + // sometimes + assertThat(addedDataFiles).hasSize(parallelism); + // verify there is no overlap in min-max stats range + if (parallelism == 2) { + assertIdColumnStatsNoRangeOverlap(addedDataFiles.get(0), addedDataFiles.get(1)); + } + } + } + + private BoundedTestSource createRangeDistributionBoundedSource( + List> rowsPerCheckpoint) { + return new BoundedTestSource<>(rowsPerCheckpoint); + } + + private List> createCharRows(int numOfCheckpoints, int countPerChar) { + List> rowsPerCheckpoint = Lists.newArrayListWithCapacity(numOfCheckpoints); + for (int checkpointId = 0; checkpointId < numOfCheckpoints; ++checkpointId) { + List rows = Lists.newArrayListWithCapacity(26 * countPerChar); + for (int j = 0; j < countPerChar; ++j) { + for (char c = 'a'; c <= 'z'; ++c) { + rows.add(Row.of(1, String.valueOf(c))); + } + } + + rowsPerCheckpoint.add(rows); + } + + return rowsPerCheckpoint; + } + + private List> createIntRows(int numOfCheckpoints, int maxId) { + List> rowsPerCheckpoint = Lists.newArrayListWithCapacity(numOfCheckpoints); + for (int checkpointId = 0; checkpointId < numOfCheckpoints; ++checkpointId) { + List rows = Lists.newArrayListWithCapacity(maxId); + for (int j = 0; j < maxId; ++j) { + // fixed value "a" for the data (possible partition column) + rows.add(Row.of(j, "a")); + } + + rowsPerCheckpoint.add(rows); + } + + return rowsPerCheckpoint; + } + + private void assertIdColumnStatsNoRangeOverlap(DataFile file1, DataFile file2) { + // id column has fieldId 1 + int file1LowerBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file1.lowerBounds().get(1)); + int file1UpperBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file1.upperBounds().get(1)); + int file2LowerBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file2.lowerBounds().get(1)); + int file2UpperBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file2.upperBounds().get(1)); + + if (file1LowerBound < file2LowerBound) { + assertThat(file1UpperBound).isLessThanOrEqualTo(file2LowerBound); + } else { + assertThat(file2UpperBound).isLessThanOrEqualTo(file1LowerBound); + } + } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java new file mode 100644 index 000000000000..a5799288b5e3 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java @@ -0,0 +1,255 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.apache.iceberg.expressions.Expressions.bucket; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.connector.datagen.source.DataGeneratorSource; +import org.apache.flink.connector.datagen.source.GeneratorFunction; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.RegisterExtension; + +/** + * Test range distribution with bucketing partition column. Compared to hash distribution, range + * distribution is more general to handle bucketing column while achieving even distribution of + * traffic to writer tasks. + * + *
    + *
  • keyBy on low cardinality (e.g. + * 60) may not achieve balanced data distribution. + *
  • number of buckets (e.g. 60) is not divisible by the writer parallelism (e.g. 40). + *
  • number of buckets (e.g. 60) is smaller than the writer parallelism (e.g. 120). + *
+ */ +@Timeout(value = 30) +@Disabled // https://github.com/apache/iceberg/pull/11305#issuecomment-2415207097 +public class TestFlinkIcebergSinkRangeDistributionBucketing { + private static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = + new Configuration() + // disable classloader check as Avro may cache class/object in the serializers. + .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); + + // max supported parallelism is 16 (= 4 x 4) + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(4) + .setNumberSlotsPerTaskManager(4) + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) + .build()); + + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + private static final int NUM_BUCKETS = 4; + private static final int NUM_OF_CHECKPOINTS = 6; + private static final int ROW_COUNT_PER_CHECKPOINT = 200; + private static final Schema SCHEMA = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "uuid", Types.UUIDType.get()), + Types.NestedField.optional(3, "data", Types.StringType.get())); + private static final PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA).hour("ts").bucket("uuid", NUM_BUCKETS).build(); + private static final RowType ROW_TYPE = FlinkSchemaUtil.convert(SCHEMA); + + private TableLoader tableLoader; + private Table table; + + @BeforeEach + public void before() throws IOException { + this.tableLoader = CATALOG_EXTENSION.tableLoader(); + this.table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SCHEMA, + SPEC, + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, FileFormat.PARQUET.name())); + + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + + // Assuming ts is on ingestion/processing time. Writer only writes to 1 or 2 hours concurrently. + // Only sort on the bucket column to avoid each writer task writes to 60 buckets/files + // concurrently. + table.replaceSortOrder().asc(bucket("uuid", NUM_BUCKETS)).commit(); + } + + @AfterEach + public void after() throws Exception { + CATALOG_EXTENSION.catalog().dropTable(TestFixtures.TABLE_IDENTIFIER); + } + + /** number of buckets 4 matches writer parallelism of 4 */ + @Test + public void testBucketNumberEqualsToWriterParallelism() throws Exception { + testParallelism(4); + } + + /** number of buckets 4 is less than writer parallelism of 6 */ + @Test + public void testBucketNumberLessThanWriterParallelismNotDivisible() throws Exception { + testParallelism(6); + } + + /** number of buckets 4 is less than writer parallelism of 8 */ + @Test + public void testBucketNumberLessThanWriterParallelismDivisible() throws Exception { + testParallelism(8); + } + + /** number of buckets 4 is greater than writer parallelism of 3 */ + @Test + public void testBucketNumberHigherThanWriterParallelismNotDivisible() throws Exception { + testParallelism(3); + } + + /** number of buckets 4 is greater than writer parallelism of 2 */ + @Test + public void testBucketNumberHigherThanWriterParallelismDivisible() throws Exception { + testParallelism(2); + } + + private void testParallelism(int parallelism) throws Exception { + try (StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism)) { + + DataGeneratorSource generatorSource = + new DataGeneratorSource<>( + new RowGenerator(), + ROW_COUNT_PER_CHECKPOINT * NUM_OF_CHECKPOINTS, + RateLimiterStrategy.perCheckpoint(ROW_COUNT_PER_CHECKPOINT), + FlinkCompatibilityUtil.toTypeInfo(ROW_TYPE)); + DataStream dataStream = + env.fromSource(generatorSource, WatermarkStrategy.noWatermarks(), "Data Generator"); + + FlinkSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the oldest snapshot to the newest snapshot + List snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Source rate limit per checkpoint cycle may not be super precise. + // There could be more checkpoint cycles and commits than planned. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(NUM_OF_CHECKPOINTS); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + for (Snapshot snapshot : rangePartitionedCycles) { + List addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + assertThat(addedDataFiles) + .hasSizeLessThanOrEqualTo(maxAddedDataFilesPerCheckpoint(parallelism)); + } + } + } + + /** + * Traffic is not perfectly balanced across all buckets in the small sample size Range + * distribution of the bucket id may cross subtask boundary. Hence the number of committed data + * files per checkpoint maybe larger than writer parallelism or the number of buckets. But it + * should not be more than the sum of those two. Without range distribution, the number of data + * files per commit can be 4x of parallelism (as the number of buckets is 4). + */ + private int maxAddedDataFilesPerCheckpoint(int parallelism) { + return NUM_BUCKETS + parallelism; + } + + private static class RowGenerator implements GeneratorFunction { + // use constant timestamp so that all rows go to the same hourly partition + private final long ts = System.currentTimeMillis(); + + @Override + public RowData map(Long index) throws Exception { + // random uuid should result in relatively balanced distribution across buckets + UUID uuid = UUID.randomUUID(); + ByteBuffer uuidByteBuffer = ByteBuffer.allocate(16); + uuidByteBuffer.putLong(uuid.getMostSignificantBits()); + uuidByteBuffer.putLong(uuid.getLeastSignificantBits()); + return GenericRowData.of( + TimestampData.fromEpochMillis(ts), + uuidByteBuffer.array(), + StringData.fromString("row-" + index)); + } + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java index 577c54976b9a..b283b8390a2b 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -30,6 +30,7 @@ import org.apache.flink.types.Row; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.DistributionMode; import org.apache.iceberg.FileFormat; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.ParameterizedTestExtension; @@ -184,11 +185,21 @@ public void testUpsertModeCheck() throws Exception { .hasMessage( "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); - assertThatThrownBy( - () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) - .isInstanceOf(IllegalStateException.class) - .hasMessage( - "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + if (writeDistributionMode.equals(DistributionMode.RANGE.modeName()) && !partitioned) { + // validation error thrown from distributeDataStream + assertThatThrownBy( + () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Invalid write distribution mode: range. Need to define sort order or partition spec."); + } else { + // validation error thrown from appendWriter + assertThatThrownBy( + () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + } } @TestTemplate diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java index fc33c2fea5e6..da4ac2ada677 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java @@ -48,30 +48,30 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.StructLikeSet; -public class TestFlinkIcebergSinkV2Base { +class TestFlinkIcebergSinkV2Base { - protected static final int FORMAT_V2 = 2; - protected static final TypeInformation ROW_TYPE_INFO = + static final int FORMAT_V2 = 2; + static final TypeInformation ROW_TYPE_INFO = new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); - protected static final int ROW_ID_POS = 0; - protected static final int ROW_DATA_POS = 1; + static final int ROW_ID_POS = 0; + static final int ROW_DATA_POS = 1; - protected TableLoader tableLoader; - protected Table table; - protected StreamExecutionEnvironment env; + TableLoader tableLoader; + Table table; + StreamExecutionEnvironment env; @Parameter(index = 0) - protected FileFormat format; + FileFormat format; @Parameter(index = 1) - protected int parallelism = 1; + int parallelism = 1; @Parameter(index = 2) - protected boolean partitioned; + boolean partitioned; @Parameter(index = 3) - protected String writeDistributionMode; + String writeDistributionMode; @Parameters(name = "FileFormat={0}, Parallelism={1}, Partitioned={2}, WriteDistributionMode={3}") public static Object[][] parameters() { @@ -91,14 +91,14 @@ public static Object[][] parameters() { }; } - protected static final Map ROW_KIND_MAP = + static final Map ROW_KIND_MAP = ImmutableMap.of( "+I", RowKind.INSERT, "-D", RowKind.DELETE, "-U", RowKind.UPDATE_BEFORE, "+U", RowKind.UPDATE_AFTER); - protected Row row(String rowKind, int id, String data) { + Row row(String rowKind, int id, String data) { RowKind kind = ROW_KIND_MAP.get(rowKind); if (kind == null) { throw new IllegalArgumentException("Unknown row kind: " + rowKind); @@ -107,7 +107,7 @@ protected Row row(String rowKind, int id, String data) { return Row.ofKind(kind, id, data); } - protected void testUpsertOnIdDataKey(String branch) throws Exception { + void testUpsertOnIdDataKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of(row("+I", 1, "aaa"), row("+U", 1, "aaa"), row("+I", 2, "bbb")), @@ -128,7 +128,7 @@ protected void testUpsertOnIdDataKey(String branch) throws Exception { branch); } - protected void testChangeLogOnIdDataKey(String branch) throws Exception { + void testChangeLogOnIdDataKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of( @@ -157,7 +157,7 @@ protected void testChangeLogOnIdDataKey(String branch) throws Exception { branch); } - protected void testChangeLogOnSameKey(String branch) throws Exception { + void testChangeLogOnSameKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( // Checkpoint #1 @@ -185,7 +185,7 @@ protected void testChangeLogOnSameKey(String branch) throws Exception { branch); } - protected void testChangeLogOnDataKey(String branch) throws Exception { + void testChangeLogOnDataKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of( @@ -213,7 +213,7 @@ protected void testChangeLogOnDataKey(String branch) throws Exception { branch); } - protected void testUpsertOnDataKey(String branch) throws Exception { + void testUpsertOnDataKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of(row("+I", 1, "aaa"), row("+I", 2, "aaa"), row("+I", 3, "bbb")), @@ -235,7 +235,7 @@ protected void testUpsertOnDataKey(String branch) throws Exception { branch); } - protected void testChangeLogOnIdKey(String branch) throws Exception { + void testChangeLogOnIdKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of( @@ -285,7 +285,7 @@ protected void testChangeLogOnIdKey(String branch) throws Exception { } } - protected void testUpsertOnIdKey(String branch) throws Exception { + void testUpsertOnIdKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of(row("+I", 1, "aaa"), row("+U", 1, "bbb")), @@ -321,7 +321,7 @@ protected void testUpsertOnIdKey(String branch) throws Exception { } } - protected void testChangeLogs( + void testChangeLogs( List equalityFieldColumns, KeySelector keySelector, boolean insertAsUpsert, @@ -358,11 +358,11 @@ protected void testChangeLogs( } } - protected Record record(int id, String data) { + Record record(int id, String data) { return SimpleDataUtil.createRecord(id, data); } - private List findValidSnapshots() { + List findValidSnapshots() { List validSnapshots = Lists.newArrayList(); for (Snapshot snapshot : table.snapshots()) { if (snapshot.allManifests(table.io()).stream() @@ -373,11 +373,11 @@ private List findValidSnapshots() { return validSnapshots; } - private StructLikeSet expectedRowSet(Record... records) { + StructLikeSet expectedRowSet(Record... records) { return SimpleDataUtil.expectedRowSet(table, records); } - private StructLikeSet actualRowSet(long snapshotId, String... columns) throws IOException { + StructLikeSet actualRowSet(long snapshotId, String... columns) throws IOException { table.refresh(); StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); try (CloseableIterable reader = diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java index 0b0c55f51c32..56cba8f460e2 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java @@ -43,15 +43,24 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestFlinkIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Base { @RegisterExtension - private static final HadoopCatalogExtension CATALOG_EXTENSION = + static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); - @Parameter(index = 0) - private String branch; + @Parameter(index = 4) + protected String branch; - @Parameters(name = "branch = {0}") + @Parameters( + name = + "FileFormat={0}, Parallelism={1}, Partitioned={2}, WriteDistributionMode={3}, Branch={4}") public static Object[][] parameters() { - return new Object[][] {new Object[] {"main"}, new Object[] {"testBranch"}}; + return new Object[][] { + new Object[] { + FileFormat.AVRO, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE, "main" + }, + new Object[] { + FileFormat.AVRO, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE, "testBranch" + } + }; } @BeforeEach diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java index 53b7c4c0cc91..c21c3d5cc21b 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.flink.sink; -import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; import static org.assertj.core.api.Assertions.assertThat; import java.io.File; @@ -134,7 +133,9 @@ public void testUserProvidedManifestLocation() throws IOException { String operatorId = newOperatorUniqueId(); File userProvidedFolder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); Map props = - ImmutableMap.of(FLINK_MANIFEST_LOCATION, userProvidedFolder.getAbsolutePath() + "///"); + ImmutableMap.of( + ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION, + userProvidedFolder.getAbsolutePath() + "///"); ManifestOutputFileFactory factory = new ManifestOutputFileFactory(() -> table, props, flinkJobId, operatorId, 1, 1); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java new file mode 100644 index 000000000000..f11aae1d6923 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java @@ -0,0 +1,1445 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; +import static org.apache.iceberg.flink.sink.SinkTestUtil.extractAndAssertCommittableSummary; +import static org.apache.iceberg.flink.sink.SinkTestUtil.extractAndAssertCommittableWithLineage; +import static org.apache.iceberg.flink.sink.SinkTestUtil.transformsToStreamElement; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.TaskInfo; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessageSerializer; +import org.apache.flink.streaming.api.connector.sink2.CommittableSummary; +import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; +import org.apache.flink.streaming.api.connector.sink2.SinkV2Assertions; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.runtime.operators.sink.CommitterOperatorFactory; +import org.apache.flink.streaming.runtime.streamrecord.StreamElement; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.StreamTask; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.table.data.RowData; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.GenericManifestFile; +import org.apache.iceberg.ManifestContent; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestBase; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@ExtendWith(ParameterizedTestExtension.class) +class TestIcebergCommitter extends TestBase { + private static final Logger LOG = LoggerFactory.getLogger(TestIcebergCommitter.class); + public static final String OPERATOR_ID = "flink-sink"; + @TempDir File temporaryFolder; + + @TempDir File flinkManifestFolder; + + private Table table; + + private TableLoader tableLoader; + + @Parameter(index = 1) + private Boolean isStreamingMode; + + @Parameter(index = 2) + private String branch; + + private final String jobId = "jobId"; + private final long dataFIleRowCount = 5L; + + private final TestCommittableMessageTypeSerializer committableMessageTypeSerializer = + new TestCommittableMessageTypeSerializer(); + + private final DataFile dataFileTest1 = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-1.parquet") + .withFileSizeInBytes(0) + .withMetrics( + new Metrics( + dataFIleRowCount, + null, // no column sizes + ImmutableMap.of(1, 5L), // value count + ImmutableMap.of(1, 0L), // null count + null, + ImmutableMap.of(1, longToBuffer(0L)), // lower bounds + ImmutableMap.of(1, longToBuffer(4L)) // upper bounds + )) + .build(); + + private final DataFile dataFileTest2 = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-2.parquet") + .withFileSizeInBytes(0) + .withMetrics( + new Metrics( + dataFIleRowCount, + null, // no column sizes + ImmutableMap.of(1, 5L), // value count + ImmutableMap.of(1, 0L), // null count + null, + ImmutableMap.of(1, longToBuffer(0L)), // lower bounds + ImmutableMap.of(1, longToBuffer(4L)) // upper bounds + )) + .build(); + + @SuppressWarnings("checkstyle:NestedForDepth") + @Parameters(name = "formatVersion={0} isStreaming={1}, branch={2}") + protected static List parameters() { + List parameters = Lists.newArrayList(); + for (Boolean isStreamingMode : new Boolean[] {true, false}) { + for (int formatVersion : new int[] {1, 2}) { + parameters.add(new Object[] {formatVersion, isStreamingMode, SnapshotRef.MAIN_BRANCH}); + parameters.add(new Object[] {formatVersion, isStreamingMode, "test-branch"}); + } + } + return parameters; + } + + @BeforeEach + public void before() throws Exception { + String warehouse = temporaryFolder.getAbsolutePath(); + + String tablePath = warehouse.concat("/test"); + assertThat(new File(tablePath).mkdir()).as("Should create the table path correctly.").isTrue(); + + Map props = + ImmutableMap.of( + TableProperties.FORMAT_VERSION, + String.valueOf(formatVersion), + FLINK_MANIFEST_LOCATION, + flinkManifestFolder.getAbsolutePath(), + IcebergCommitter.MAX_CONTINUOUS_EMPTY_COMMITS, + "1"); + table = SimpleDataUtil.createTable(tablePath, props, false); + tableLoader = TableLoader.fromHadoopTable(tablePath); + } + + @TestTemplate + public void testCommitTxnWithoutDataFiles() throws Exception { + IcebergCommitter committer = getCommitter(); + SimpleDataUtil.assertTableRows(table, Lists.newArrayList(), branch); + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId, -1); + + for (long i = 1; i <= 3; i++) { + Committer.CommitRequest commitRequest = + buildCommitRequestFor(jobId, i, Lists.newArrayList()); + committer.commit(Lists.newArrayList(commitRequest)); + assertMaxCommittedCheckpointId(jobId, i); + assertSnapshotSize((int) i); + } + } + + @TestTemplate + public void testMxContinuousEmptyCommits() throws Exception { + table.updateProperties().set(IcebergCommitter.MAX_CONTINUOUS_EMPTY_COMMITS, "3").commit(); + IcebergCommitter committer = getCommitter(); + for (int i = 1; i <= 9; i++) { + Committer.CommitRequest commitRequest = + buildCommitRequestFor(jobId, i, Lists.newArrayList()); + committer.commit(Lists.newArrayList(commitRequest)); + assertFlinkManifests(0); + assertSnapshotSize(i / 3); + } + } + + @TestTemplate + public void testCommitTxn() throws Exception { + IcebergCommitter committer = getCommitter(); + assertSnapshotSize(0); + List rows = Lists.newArrayListWithExpectedSize(3); + for (int i = 1; i <= 3; i++) { + RowData rowData = SimpleDataUtil.createRowData(i, "hello" + i); + DataFile dataFile = writeDataFile("data-" + i, ImmutableList.of(rowData)); + rows.add(rowData); + WriteResult writeResult = of(dataFile); + Committer.CommitRequest commitRequest = + buildCommitRequestFor(jobId, i, Lists.newArrayList(writeResult)); + committer.commit(Lists.newArrayList(commitRequest)); + assertFlinkManifests(0); + SimpleDataUtil.assertTableRows(table, ImmutableList.copyOf(rows), branch); + assertSnapshotSize(i); + assertMaxCommittedCheckpointId(jobId, i); + Map summary = SimpleDataUtil.latestSnapshot(table, branch).summary(); + assertThat(summary) + .containsEntry("flink.test", "org.apache.iceberg.flink.sink.TestIcebergCommitter") + .containsEntry("added-data-files", "1") + .containsEntry("flink.operator-id", OPERATOR_ID) + .containsEntry("flink.job-id", "jobId"); + } + } + + @TestTemplate + public void testOrderedEventsBetweenCheckpoints() throws Exception { + // It's possible that two checkpoints happen in the following orders: + // 1. snapshotState for checkpoint#1; + // 2. snapshotState for checkpoint#2; + // 3. notifyCheckpointComplete for checkpoint#1; + // 4. notifyCheckpointComplete for checkpoint#2; + + long timestamp = 0; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + + harness.open(); + + assertMaxCommittedCheckpointId(jobId, -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello"); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + + processElement(jobId, 1, harness, 1, OPERATOR_ID, dataFile1); + assertMaxCommittedCheckpointId(jobId, -1L); + + // 1. snapshotState for checkpoint#1 + long firstCheckpointId = 1; + harness.snapshot(firstCheckpointId, ++timestamp); + assertFlinkManifests(1); + + RowData row2 = SimpleDataUtil.createRowData(2, "world"); + DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); + processElement(jobId, 2, harness, 1, OPERATOR_ID, dataFile2); + assertMaxCommittedCheckpointId(jobId, -1L); + + // 2. snapshotState for checkpoint#2 + long secondCheckpointId = 2; + OperatorSubtaskState snapshot = harness.snapshot(secondCheckpointId, ++timestamp); + assertFlinkManifests(2); + + // 3. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(firstCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertMaxCommittedCheckpointId(jobId, firstCheckpointId); + assertFlinkManifests(1); + + // 4. notifyCheckpointComplete for checkpoint#2 + harness.notifyOfCompletedCheckpoint(secondCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); + assertMaxCommittedCheckpointId(jobId, secondCheckpointId); + assertFlinkManifests(0); + } + } + + @TestTemplate + public void testDisorderedEventsBetweenCheckpoints() throws Exception { + // It's possible that two checkpoints happen in the following orders: + // 1. snapshotState for checkpoint#1; + // 2. snapshotState for checkpoint#2; + // 3. notifyCheckpointComplete for checkpoint#2; + // 4. notifyCheckpointComplete for checkpoint#1; + + long timestamp = 0; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + + harness.open(); + assertMaxCommittedCheckpointId(jobId, -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello"); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + + processElement(jobId, 1, harness, 1, OPERATOR_ID, dataFile1); + assertMaxCommittedCheckpointId(jobId, -1L); + + // 1. snapshotState for checkpoint#1 + long firstCheckpointId = 1; + harness.snapshot(firstCheckpointId, ++timestamp); + assertFlinkManifests(1); + + RowData row2 = SimpleDataUtil.createRowData(2, "world"); + DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); + processElement(jobId, 2, harness, 1, OPERATOR_ID, dataFile2); + assertMaxCommittedCheckpointId(jobId, -1L); + + // 2. snapshotState for checkpoint#2 + long secondCheckpointId = 2; + harness.snapshot(secondCheckpointId, ++timestamp); + assertFlinkManifests(2); + + // 3. notifyCheckpointComplete for checkpoint#2 + harness.notifyOfCompletedCheckpoint(secondCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); + assertMaxCommittedCheckpointId(jobId, secondCheckpointId); + assertFlinkManifests(0); + + // 4. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(firstCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); + assertMaxCommittedCheckpointId(jobId, secondCheckpointId); + assertFlinkManifests(0); + } + } + + @TestTemplate + public void testSingleCommit() throws Exception { + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness = getTestHarness()) { + testHarness.open(); + + long checkpointId = 1; + + RowData row1 = SimpleDataUtil.createRowData(1, "hello1"); + DataFile dataFile1 = writeDataFile("data-1-1", ImmutableList.of(row1)); + CommittableSummary committableSummary = + processElement(jobId, checkpointId, testHarness, 1, OPERATOR_ID, dataFile1); + + // Trigger commit + testHarness.notifyOfCompletedCheckpoint(checkpointId); + + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId, 1L); + + List output = transformsToStreamElement(testHarness.getOutput()); + + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary.getNumberOfCommittables()) + .hasPendingCommittables(0); + + SinkV2Assertions.assertThat(extractAndAssertCommittableWithLineage(output.get(1))) + .hasSubtaskId(0) + .hasCheckpointId(checkpointId); + } + + table.refresh(); + Snapshot currentSnapshot = table.snapshot(branch); + + assertThat(currentSnapshot.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, "1") + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1"); + } + + /** The data was not committed in the previous job. */ + @TestTemplate + public void testStateRestoreFromPreJobWithUncommitted() throws Exception { + String jobId1 = "jobId1"; + OperatorSubtaskState snapshot; + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + long timestamp = 0; + CommittableSummary committableSummary; + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + preJobTestHarness = getTestHarness()) { + + preJobTestHarness.open(); + + committableSummary = + processElement(jobId1, checkpointId, preJobTestHarness, 1, OPERATOR_ID, dataFileTest1); + + snapshot = preJobTestHarness.snapshot(checkpointId, ++timestamp); + + assertThat(preJobTestHarness.getOutput()).isEmpty(); + } + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId1, -1L); + + String jobId2 = "jobId2"; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + restored = getTestHarness()) { + restored.setup(committableMessageTypeSerializer); + restored.initializeState(snapshot); + restored.open(); + + // Previous committables are immediately committed if possible + List output = transformsToStreamElement(restored.getOutput()); + assertThat(output).hasSize(2); + + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary.getNumberOfCommittables()) + .hasPendingCommittables(0); + + SinkV2Assertions.assertThat(extractAndAssertCommittableWithLineage(output.get(1))) + .hasCheckpointId(0L) + .hasSubtaskId(0); + + table.refresh(); + + Snapshot currentSnapshot = table.snapshot(branch); + + assertThat(currentSnapshot.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1") + .containsEntry("flink.job-id", jobId1); + + checkpointId++; + CommittableSummary committableSummary2 = + processElement(jobId2, checkpointId, restored, 1, OPERATOR_ID, dataFileTest2); + + // Trigger commit + restored.notifyOfCompletedCheckpoint(checkpointId); + + List output2 = transformsToStreamElement(restored.getOutput()); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output2.get(0))) + .hasFailedCommittables(committableSummary2.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary2.getNumberOfCommittables()) + .hasPendingCommittables(0); + + SinkV2Assertions.assertThat(extractAndAssertCommittableWithLineage(output2.get(1))) + .hasCheckpointId(0L) + .hasSubtaskId(0); + } + + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId2, 1); + + table.refresh(); + Snapshot currentSnapshot2 = table.snapshot(branch); + + assertThat(currentSnapshot2.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount * 2)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "2") + .containsEntry("flink.job-id", jobId2); + } + + /** The data was committed in the previous job. */ + @TestTemplate + public void testStateRestoreFromPreJobWithCommitted() throws Exception { + String jobId1 = "jobId1"; + OperatorSubtaskState snapshot; + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + preJobTestHarness = getTestHarness()) { + + preJobTestHarness.open(); + + CommittableSummary committableSummary = + processElement(jobId1, checkpointId, preJobTestHarness, 1, OPERATOR_ID, dataFileTest1); + + assertFlinkManifests(1); + snapshot = preJobTestHarness.snapshot(checkpointId, 2L); + // commit snapshot + preJobTestHarness.notifyOfCompletedCheckpoint(checkpointId); + + List output = transformsToStreamElement(preJobTestHarness.getOutput()); + assertThat(output).hasSize(2); + + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary.getNumberOfCommittables()) + .hasPendingCommittables(0); + + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId1, checkpointId); + } + + table.refresh(); + long preJobSnapshotId = table.snapshot(branch).snapshotId(); + + String jobId2 = "jobId2"; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + restored = getTestHarness()) { + restored.setup(); + restored.initializeState(snapshot); + restored.open(); + + // Makes sure that data committed in the previous job is available in this job + List output2 = transformsToStreamElement(restored.getOutput()); + assertThat(output2).hasSize(2); + + table.refresh(); + long restoredSnapshotId = table.snapshot(branch).snapshotId(); + + assertThat(restoredSnapshotId) + .as("The table does not generate a new snapshot without data being committed.") + .isEqualTo(preJobSnapshotId); + + assertThat(table.snapshot(branch).summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1") + .containsEntry("flink.job-id", jobId1); + + // Commit new data file + checkpointId = 1; + CommittableSummary committableSummary2 = + processElement(jobId2, checkpointId, restored, 1, OPERATOR_ID, dataFileTest2); + + // Trigger commit + restored.notifyOfCompletedCheckpoint(checkpointId); + + List output3 = transformsToStreamElement(restored.getOutput()); + assertThat(output3).hasSize(4); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output3.get(0))) + .hasFailedCommittables(committableSummary2.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary2.getNumberOfCommittables()) + .hasPendingCommittables(0); + } + + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId2, 1L); + + table.refresh(); + Snapshot currentSnapshot2 = table.snapshot(branch); + assertThat(Long.parseLong(currentSnapshot2.summary().get(SnapshotSummary.TOTAL_RECORDS_PROP))) + .isEqualTo(dataFIleRowCount * 2); + + assertThat(currentSnapshot2.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount * 2)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "2") + .containsEntry("flink.job-id", jobId2); + } + + @TestTemplate + public void testStateRestoreFromCurrJob() throws Exception { + String jobId1 = "jobId1"; + CommittableSummary committableSummary; + OperatorSubtaskState snapshot; + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness = getTestHarness()) { + + testHarness.open(); + + committableSummary = + processElement(jobId1, checkpointId, testHarness, 1, OPERATOR_ID, dataFileTest1); + snapshot = testHarness.snapshot(checkpointId, 2L); + + assertThat(testHarness.getOutput()).isEmpty(); + } + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId1, -1L); + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + restored = getTestHarness()) { + + restored.setup(committableMessageTypeSerializer); + + restored.initializeState(snapshot); + restored.open(); + + // Previous committables are immediately committed if possible + List output = transformsToStreamElement(restored.getOutput()); + assertThat(output).hasSize(2); + + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary.getNumberOfCommittables()) + .hasPendingCommittables(0); + + table.refresh(); + Snapshot currentSnapshot = table.snapshot(branch); + + assertThat(currentSnapshot.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1") + .containsEntry("flink.job-id", jobId1); + + String jobId2 = "jobId2"; + checkpointId = 1; + CommittableSummary committableSummary2 = + processElement(jobId2, checkpointId, restored, 1, OPERATOR_ID, dataFileTest2); + + // Trigger commit + restored.notifyOfCompletedCheckpoint(checkpointId); + + List output2 = transformsToStreamElement(restored.getOutput()); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output2.get(0))) + .hasFailedCommittables(committableSummary2.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary2.getNumberOfCommittables()) + .hasPendingCommittables(0); + restored.close(); + + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId2, 1L); + + table.refresh(); + Snapshot currentSnapshot2 = table.snapshot(branch); + assertThat(currentSnapshot2.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount * 2)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "2") + .containsEntry("flink.job-id", jobId2); + } + } + + @TestTemplate + public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Exception { + // We've two steps in checkpoint: 1. snapshotState(ckp); 2. notifyCheckpointComplete(ckp). + // The Flink job should be able to restore from a checkpoint with only step#1 finished. + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + long timestamp = 0; + OperatorSubtaskState snapshot; + List expectedRows = Lists.newArrayList(); + + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId, -1L); + + RowData row = SimpleDataUtil.createRowData(1, "hello"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-1", ImmutableList.of(row)); + processElement(jobId, checkpointId, harness, 1, operatorId.toString(), dataFile); + + snapshot = harness.snapshot(++checkpointId, ++timestamp); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(), branch); + assertMaxCommittedCheckpointId(jobId, -1L); + assertFlinkManifests(1); + } + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + + harness.getStreamConfig().setOperatorID(operatorId); + harness.initializeState(snapshot); + harness.open(); + + // All flink manifests should be cleaned because it has committed the unfinished iceberg + // transaction. + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertMaxCommittedCheckpointId(jobId, operatorId.toString(), 0L); + + harness.snapshot(++checkpointId, ++timestamp); + // Did not write any new record, so it won't generate new manifest. + assertFlinkManifests(0); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(1); + + assertMaxCommittedCheckpointId(jobId, operatorId.toString(), 0); + + RowData row = SimpleDataUtil.createRowData(2, "world"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); + processElement(jobId, checkpointId, harness, 1, operatorId.toString(), dataFile); + + snapshot = harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(0); + } + + // Redeploying flink job from external checkpoint. + JobID newJobId = new JobID(); + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + harness.getStreamConfig().setOperatorID(operatorId); + harness.initializeState(snapshot); + harness.open(); + + // All flink manifests should be cleaned because it has committed the unfinished iceberg + // transaction. + assertFlinkManifests(0); + + assertMaxCommittedCheckpointId(newJobId.toString(), operatorId.toString(), -1); + assertMaxCommittedCheckpointId(jobId, operatorId.toString(), 2); + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(2); + + RowData row = SimpleDataUtil.createRowData(3, "foo"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-3", ImmutableList.of(row)); + processElement( + newJobId.toString(), checkpointId, harness, 1, operatorId.toString(), dataFile); + + harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(3); + assertMaxCommittedCheckpointId(newJobId.toString(), operatorId.toString(), 3); + } + } + + @TestTemplate + public void testStartAnotherJobToWriteSameTable() throws Exception { + long checkpointId = 1; + long timestamp = 0; + + List rows = Lists.newArrayList(); + List tableRows = Lists.newArrayList(); + + JobID oldJobId = new JobID(); + OperatorID oldOperatorId; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + + harness.open(); + oldOperatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(oldJobId.toString(), oldOperatorId.toString(), -1L); + + for (int i = 1; i <= 3; i++) { + rows.add(SimpleDataUtil.createRowData(i, "hello" + i)); + tableRows.addAll(rows); + + DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); + processElement( + oldJobId.toString(), ++checkpointId, harness, 1, oldOperatorId.toString(), dataFile); + harness.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(i); + assertMaxCommittedCheckpointId(oldJobId.toString(), oldOperatorId.toString(), checkpointId); + } + } + + // The new started job will start with checkpoint = 1 again. + checkpointId = 1; + JobID newJobId = new JobID(); + OperatorID newOperatorId; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + harness.open(); + newOperatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(3); + assertMaxCommittedCheckpointId(oldJobId.toString(), oldOperatorId.toString(), 4); + assertMaxCommittedCheckpointId(newJobId.toString(), newOperatorId.toString(), -1); + + rows.add(SimpleDataUtil.createRowData(2, "world")); + tableRows.addAll(rows); + + DataFile dataFile = writeDataFile("data-new-1", rows); + processElement( + newJobId.toString(), checkpointId, harness, 1, newOperatorId.toString(), dataFile); + harness.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(4); + assertMaxCommittedCheckpointId(newJobId.toString(), newOperatorId.toString(), checkpointId); + } + } + + @TestTemplate + public void testMultipleJobsWriteSameTable() throws Exception { + long timestamp = 0; + List tableRows = Lists.newArrayList(); + + JobID[] jobs = new JobID[] {new JobID(), new JobID(), new JobID()}; + OperatorID[] operatorIds = + new OperatorID[] {new OperatorID(), new OperatorID(), new OperatorID()}; + for (int i = 0; i < 20; i++) { + int jobIndex = i % 3; + int checkpointId = i / 3; + JobID jobID = jobs[jobIndex]; + OperatorID operatorId = operatorIds[jobIndex]; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + harness.getStreamConfig().setOperatorID(operatorId); + + harness.open(); + + assertSnapshotSize(i); + assertMaxCommittedCheckpointId( + jobID.toString(), operatorId.toString(), checkpointId == 0 ? -1 : checkpointId - 1); + + List rows = Lists.newArrayList(SimpleDataUtil.createRowData(i, "word-" + i)); + tableRows.addAll(rows); + + DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); + + processElement(jobID.toString(), checkpointId, harness, 1, operatorId.toString(), dataFile); + + harness.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(i + 1); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId.toString(), checkpointId); + } + } + } + + @TestTemplate + public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + long timestamp = 0; + List expectedRows = Lists.newArrayList(); + OperatorSubtaskState snapshot1; + OperatorSubtaskState snapshot2; + + JobID jobID = new JobID(); + OperatorID operatorId1 = new OperatorID(); + OperatorID operatorId2 = new OperatorID(); + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness1 = getTestHarness()) { + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness2 = getTestHarness()) { + harness1.getStreamConfig().setOperatorID(operatorId1); + harness1.setup(); + harness1.open(); + harness2.getStreamConfig().setOperatorID(operatorId2); + harness2.setup(); + harness2.open(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId1.toString(), -1L); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId2.toString(), -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello1"); + expectedRows.add(row1); + DataFile dataFile1 = writeDataFile("data-1-1", ImmutableList.of(row1)); + processElement( + jobID.toString(), checkpointId, harness1, 1, operatorId1.toString(), dataFile1); + + snapshot1 = harness1.snapshot(checkpointId, ++timestamp); + + RowData row2 = SimpleDataUtil.createRowData(1, "hello2"); + expectedRows.add(row2); + DataFile dataFile2 = writeDataFile("data-1-2", ImmutableList.of(row2)); + processElement( + jobID.toString(), checkpointId, harness2, 1, operatorId2.toString(), dataFile2); + + snapshot2 = harness2.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(2); + + // Only notify one of the committers + harness1.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(1); + + // Only the first row is committed at this point + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId1.toString(), checkpointId); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId2.toString(), -1); + } + } + + // Restore from the given snapshot + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness1 = getTestHarness(); + OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness2 = getTestHarness()) { + harness1.getStreamConfig().setOperatorID(operatorId1); + harness1.setup(); + harness1.initializeState(snapshot1); + harness1.open(); + + harness2.getStreamConfig().setOperatorID(operatorId2); + harness2.setup(); + harness2.initializeState(snapshot2); + harness2.open(); + + // All flink manifests should be cleaned because it has committed the unfinished iceberg + // transaction. + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId1.toString(), checkpointId); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId2.toString(), checkpointId); + + RowData row1 = SimpleDataUtil.createRowData(2, "world1"); + expectedRows.add(row1); + DataFile dataFile1 = writeDataFile("data-2-1", ImmutableList.of(row1)); + + checkpointId++; + processElement( + jobID.toString(), checkpointId, harness1, 1, operatorId1.toString(), dataFile1); + + harness1.snapshot(checkpointId, ++timestamp); + + RowData row2 = SimpleDataUtil.createRowData(2, "world2"); + expectedRows.add(row2); + DataFile dataFile2 = writeDataFile("data-2-2", ImmutableList.of(row2)); + processElement( + jobID.toString(), checkpointId, harness2, 1, operatorId2.toString(), dataFile2); + + harness2.snapshot(checkpointId, ++timestamp); + + assertFlinkManifests(2); + + harness1.notifyOfCompletedCheckpoint(checkpointId); + harness2.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(4); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId1.toString(), checkpointId); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId2.toString(), checkpointId); + } + } + + @TestTemplate + public void testFlinkManifests() throws Exception { + long timestamp = 0; + long checkpoint = 1; + + JobID jobID = new JobID(); + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertMaxCommittedCheckpointId(jobID.toString(), operatorId.toString(), -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello"); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + + // harness.processElement(of(dataFile1), ++timestamp); + processElement(jobID.toString(), checkpoint, harness, 1, operatorId.toString(), dataFile1); + + assertMaxCommittedCheckpointId(jobID.toString(), operatorId.toString(), -1L); + + // 1. snapshotState for checkpoint#1 + harness.snapshot(checkpoint, ++timestamp); + List manifestPaths = assertFlinkManifests(1); + Path manifestPath = manifestPaths.get(0); + assertThat(manifestPath.getFileName()) + .asString() + .isEqualTo( + String.format("%s-%s-%05d-%d-%d-%05d.avro", jobID, operatorId, 0, 0, checkpoint, 1)); + // + // 2. Read the data files from manifests and assert. + List dataFiles = + FlinkManifestUtil.readDataFiles( + createTestingManifestFile(manifestPath), table.io(), table.specs()); + assertThat(dataFiles).hasSize(1); + TestHelpers.assertEquals(dataFile1, dataFiles.get(0)); + + // 3. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(checkpoint); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId.toString(), checkpoint); + assertFlinkManifests(0); + } + } + + @TestTemplate + public void testHandleEndInput() throws Exception { + assumeThat(isStreamingMode).as("Only support batch mode").isFalse(); + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness = getTestHarness()) { + + testHarness.open(); + + long checkpointId = Long.MAX_VALUE; + processElement(jobId, checkpointId, testHarness, 1, OPERATOR_ID, dataFileTest1); + + testHarness.endInput(); + + assertMaxCommittedCheckpointId(jobId, OPERATOR_ID, Long.MAX_VALUE); + + List output = transformsToStreamElement(testHarness.getOutput()); + assertThat(output).hasSize(2); + + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasCheckpointId(checkpointId) + .hasPendingCommittables(0) + .hasOverallCommittables(1) + .hasFailedCommittables(0); + + // endInput is idempotent + testHarness.endInput(); + assertThat(testHarness.getOutput()).hasSize(2); + } + } + + @TestTemplate + public void testDeleteFiles() throws Exception { + + assumeThat(formatVersion).as("Only support delete in format v2").isGreaterThanOrEqualTo(2); + + FileAppenderFactory appenderFactory = createDeletableAppenderFactory(); + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness = getTestHarness()) { + + testHarness.open(); + + long checkpointId = 1; + RowData row1 = SimpleDataUtil.createInsert(1, "aaa"); + DataFile dataFile1 = writeDataFile("data-file-1", ImmutableList.of(row1)); + processElement(jobId, checkpointId, testHarness, 1, OPERATOR_ID, dataFile1); + + // testHarness.snapshot(checkpointId, 0); + testHarness.notifyOfCompletedCheckpoint(checkpointId); + + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId, checkpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + + List output = transformsToStreamElement(testHarness.getOutput()); + assertThat(output).hasSize(2); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasCheckpointId(checkpointId) + .hasPendingCommittables(0) + .hasOverallCommittables(1) + .hasFailedCommittables(0); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + + // The 2. commit + checkpointId = 2; + RowData row2 = SimpleDataUtil.createInsert(2, "bbb"); + DataFile dataFile2 = writeDataFile("data-file-2", ImmutableList.of(row2)); + + RowData row3 = SimpleDataUtil.createInsert(3, "ccc"); + DataFile dataFile3 = writeDataFile("data-file-3", ImmutableList.of(row3)); + processElement(jobId, checkpointId, testHarness, 2, OPERATOR_ID, dataFile2, dataFile3); + + // testHarness.snapshot(checkpointId, 1); + testHarness.notifyOfCompletedCheckpoint(checkpointId); + + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId, checkpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2, row3), branch); + + List output2 = transformsToStreamElement(testHarness.getOutput()); + assertThat(output2).hasSize(2 + 2); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output2.get(2))) + .hasCheckpointId(checkpointId) + .hasPendingCommittables(0) + .hasOverallCommittables(1) + .hasFailedCommittables(0); + + // The 3. commit + checkpointId = 3; + RowData delete1 = SimpleDataUtil.createDelete(1, "aaa"); + DeleteFile deleteFile1 = + writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete1)); + RowData row4 = SimpleDataUtil.createInsert(4, "ddd"); + DataFile dataFile4 = writeDataFile("data-file-4", ImmutableList.of(row4)); + + RowData row5 = SimpleDataUtil.createInsert(5, "eee"); + DataFile dataFile5 = writeDataFile("data-file-5", ImmutableList.of(row5)); + WriteResult withRecord4 = + WriteResult.builder() + .addDataFiles(dataFile4, dataFile5) + .addDeleteFiles(deleteFile1) + .build(); + processElement(withRecord4, jobId, checkpointId, testHarness, 2, OPERATOR_ID); + + // testHarness.snapshot(checkpointId, 3); + testHarness.notifyOfCompletedCheckpoint(checkpointId); + + assertSnapshotSize(3); + assertMaxCommittedCheckpointId(jobId, checkpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row2, row3, row4, row5), branch); + + List output3 = transformsToStreamElement(testHarness.getOutput()); + assertThat(output3).hasSize(2 + 2 + 2); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output3.get(4))) + .hasCheckpointId(checkpointId) + .hasPendingCommittables(0) + .hasOverallCommittables(1) + .hasFailedCommittables(0); + } + } + + private ManifestFile createTestingManifestFile(Path manifestPath) { + return new GenericManifestFile( + manifestPath.toAbsolutePath().toString(), + manifestPath.toFile().length(), + 0, + ManifestContent.DATA, + 0, + 0, + 0L, + 0, + 0, + 0, + 0, + 0, + 0, + null, + null); + } + + private IcebergWriteAggregator buildIcebergWriteAggregator(String myJobId, String operatorId) { + IcebergWriteAggregator icebergWriteAggregator = spy(new IcebergWriteAggregator(tableLoader)); + StreamTask ctx = mock(StreamTask.class); + Environment env = mock(Environment.class); + StreamingRuntimeContext streamingRuntimeContext = mock(StreamingRuntimeContext.class); + TaskInfo taskInfo = mock(TaskInfo.class); + JobID myJobID = mock(JobID.class); + OperatorID operatorID = mock(OperatorID.class); + doReturn(myJobId).when(myJobID).toString(); + doReturn(myJobID).when(env).getJobID(); + doReturn(env).when(ctx).getEnvironment(); + doReturn(ctx).when(icebergWriteAggregator).getContainingTask(); + doReturn(operatorId).when(operatorID).toString(); + doReturn(operatorID).when(icebergWriteAggregator).getOperatorID(); + doReturn(0).when(taskInfo).getAttemptNumber(); + doReturn(taskInfo).when(streamingRuntimeContext).getTaskInfo(); + doReturn(streamingRuntimeContext).when(icebergWriteAggregator).getRuntimeContext(); + + try { + icebergWriteAggregator.open(); + } catch (Exception e) { + throw new RuntimeException(e); + } + return icebergWriteAggregator; + } + + private CommittableSummary processElement( + WriteResult withRecord, + String myJobId, + long checkpointId, + OneInputStreamOperatorTestHarness testHarness, + int subTaskId, + String operatorId) + throws Exception { + + IcebergCommittable commit = + new IcebergCommittable( + buildIcebergWriteAggregator(myJobId, operatorId) + .writeToManifest(Lists.newArrayList(withRecord), checkpointId), + myJobId, + operatorId, + checkpointId); + + CommittableSummary committableSummary = + new CommittableSummary<>(subTaskId, 1, checkpointId, 1, 1, 0); + testHarness.processElement(new StreamRecord<>(committableSummary)); + + CommittableWithLineage committable = + new CommittableWithLineage<>(commit, checkpointId, subTaskId); + testHarness.processElement(new StreamRecord<>(committable)); + + return committableSummary; + } + + private CommittableSummary processElement( + String myJobID, + long checkpointId, + OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness, + int subTaskId, + String operatorId, + DataFile... dataFile) + throws Exception { + WriteResult withRecord = WriteResult.builder().addDataFiles(dataFile).build(); + return processElement(withRecord, myJobID, checkpointId, testHarness, subTaskId, operatorId); + } + + private FileAppenderFactory createDeletableAppenderFactory() { + int[] equalityFieldIds = + new int[] { + table.schema().findField("id").fieldId(), table.schema().findField("data").fieldId() + }; + return new FlinkAppenderFactory( + table, + table.schema(), + FlinkSchemaUtil.convert(table.schema()), + table.properties(), + table.spec(), + equalityFieldIds, + table.schema(), + null); + } + + private List assertFlinkManifests(int expectedCount) throws IOException { + List manifests = + Files.list(flinkManifestFolder.toPath()) + .filter(p -> !p.toString().endsWith(".crc")) + .collect(Collectors.toList()); + assertThat(manifests).hasSize(expectedCount); + return manifests; + } + + private DataFile writeDataFile(String filename, List rows) throws IOException { + return SimpleDataUtil.writeFile( + table, + table.schema(), + table.spec(), + new Configuration(), + table.location(), + FileFormat.PARQUET.addExtension(filename), + rows); + } + + private DeleteFile writeEqDeleteFile( + FileAppenderFactory appenderFactory, String filename, List deletes) + throws IOException { + return SimpleDataUtil.writeEqDeleteFile( + table, FileFormat.PARQUET, filename, appenderFactory, deletes); + } + + private OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + getTestHarness() throws Exception { + IcebergSink sink = + IcebergSink.forRowData(null).table(table).toBranch(branch).tableLoader(tableLoader).build(); + + OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness = + new OneInputStreamOperatorTestHarness<>( + new CommitterOperatorFactory<>(sink, !isStreamingMode, true)); + testHarness.setup(committableMessageTypeSerializer); + return testHarness; + } + + // ------------------------------- Utility Methods -------------------------------- + + private IcebergCommitter getCommitter() { + IcebergFilesCommitterMetrics metric = mock(IcebergFilesCommitterMetrics.class); + return new IcebergCommitter( + tableLoader, + branch, + Collections.singletonMap("flink.test", TestIcebergCommitter.class.getName()), + false, + 10, + "sinkId", + metric); + } + + private Committer.CommitRequest buildCommitRequestFor( + String myJobID, long checkpoint, Collection writeResults) throws IOException { + IcebergCommittable commit = + new IcebergCommittable( + buildIcebergWriteAggregator(myJobID, OPERATOR_ID) + .writeToManifest(writeResults, checkpoint), + myJobID, + OPERATOR_ID, + checkpoint); + + CommittableWithLineage committableWithLineage = + new CommittableWithLineage(commit, checkpoint, 1); + Committer.CommitRequest commitRequest = mock(Committer.CommitRequest.class); + + doReturn(committableWithLineage.getCommittable()).when(commitRequest).getCommittable(); + + return commitRequest; + } + + private WriteResult of(DataFile dataFile) { + return WriteResult.builder().addDataFiles(dataFile).build(); + } + + private void assertMaxCommittedCheckpointId(String myJobID, String operatorId, long expectedId) { + table.refresh(); + long actualId = SinkUtil.getMaxCommittedCheckpointId(table, myJobID, operatorId, branch); + assertThat(actualId).isEqualTo(expectedId); + } + + private void assertMaxCommittedCheckpointId(String myJobID, long expectedId) { + assertMaxCommittedCheckpointId(myJobID, OPERATOR_ID, expectedId); + } + + private void assertSnapshotSize(int expectedSnapshotSize) { + table.refresh(); + assertThat(table.snapshots()).hasSize(expectedSnapshotSize); + } + + private static ByteBuffer longToBuffer(long value) { + return ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN).putLong(0, value); + } + + private static class TestCommittableMessageTypeSerializer + extends TypeSerializer> { + + CommittableMessageSerializer serializer = + new CommittableMessageSerializer<>(new IcebergCommittableSerializer()); + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer> duplicate() { + return null; + } + + @Override + public CommittableMessage createInstance() { + return null; + } + + @Override + public CommittableMessage copy( + CommittableMessage from) { + return from; + } + + @Override + public CommittableMessage copy( + CommittableMessage from, CommittableMessage reuse) { + return from; + } + + @Override + public int getLength() { + return 0; + } + + @Override + public void serialize(CommittableMessage record, DataOutputView target) + throws IOException { + byte[] serialize = serializer.serialize(record); + target.writeInt(serialize.length); + target.write(serialize); + } + + @Override + public CommittableMessage deserialize(DataInputView source) + throws IOException { + int length = source.readInt(); + byte[] bytes = new byte[length]; + source.read(bytes); + return serializer.deserialize(1, bytes); + } + + @Override + public CommittableMessage deserialize( + CommittableMessage reuse, DataInputView source) throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + CommittableMessage deserialize = deserialize(source); + serialize(deserialize, target); + } + + @Override + public boolean equals(Object obj) { + return false; + } + + @Override + public int hashCode() { + return 0; + } + + @Override + public TypeSerializerSnapshot> snapshotConfiguration() { + return null; + } + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java index 948c7b31430c..7808771d9887 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java @@ -129,7 +129,8 @@ public void testCommitTxnWithoutDataFiles() throws Exception { long timestamp = 0; JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -139,8 +140,7 @@ public void testCommitTxnWithoutDataFiles() throws Exception { assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // It's better to advance the max-committed-checkpoint-id in iceberg snapshot, so that the - // future flink job - // failover won't fail. + // future flink job failover won't fail. for (int i = 1; i <= 3; i++) { harness.snapshot(++checkpointId, ++timestamp); assertFlinkManifests(0); @@ -161,7 +161,8 @@ public void testMaxContinuousEmptyCommits() throws Exception { JobID jobId = new JobID(); long checkpointId = 0; long timestamp = 0; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); @@ -176,8 +177,8 @@ public void testMaxContinuousEmptyCommits() throws Exception { } } - private WriteResult of(DataFile dataFile) { - return WriteResult.builder().addDataFiles(dataFile).build(); + private FlinkWriteResult of(long checkpointId, DataFile dataFile) { + return new FlinkWriteResult(checkpointId, WriteResult.builder().addDataFiles(dataFile).build()); } @TestTemplate @@ -193,7 +194,8 @@ public void testCommitTxn() throws Exception { JobID jobID = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobID)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobID)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -204,7 +206,7 @@ public void testCommitTxn() throws Exception { for (int i = 1; i <= 3; i++) { RowData rowData = SimpleDataUtil.createRowData(i, "hello" + i); DataFile dataFile = writeDataFile("data-" + i, ImmutableList.of(rowData)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(i, dataFile), ++timestamp); rows.add(rowData); harness.snapshot(i, ++timestamp); @@ -233,7 +235,8 @@ public void testOrderedEventsBetweenCheckpoints() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -243,21 +246,21 @@ public void testOrderedEventsBetweenCheckpoints() throws Exception { RowData row1 = SimpleDataUtil.createRowData(1, "hello"); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + long firstCheckpointId = 1; + harness.processElement(of(firstCheckpointId, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 - long firstCheckpointId = 1; harness.snapshot(firstCheckpointId, ++timestamp); assertFlinkManifests(1); RowData row2 = SimpleDataUtil.createRowData(2, "world"); DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); - harness.processElement(of(dataFile2), ++timestamp); + long secondCheckpointId = 2; + harness.processElement(of(secondCheckpointId, dataFile2), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 2. snapshotState for checkpoint#2 - long secondCheckpointId = 2; harness.snapshot(secondCheckpointId, ++timestamp); assertFlinkManifests(2); @@ -286,7 +289,8 @@ public void testDisorderedEventsBetweenCheckpoints() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -296,21 +300,21 @@ public void testDisorderedEventsBetweenCheckpoints() throws Exception { RowData row1 = SimpleDataUtil.createRowData(1, "hello"); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + long firstCheckpointId = 1; + harness.processElement(of(firstCheckpointId, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 - long firstCheckpointId = 1; harness.snapshot(firstCheckpointId, ++timestamp); assertFlinkManifests(1); RowData row2 = SimpleDataUtil.createRowData(2, "world"); DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); - harness.processElement(of(dataFile2), ++timestamp); + long secondCheckpointId = 2; + harness.processElement(of(secondCheckpointId, dataFile2), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 2. snapshotState for checkpoint#2 - long secondCheckpointId = 2; harness.snapshot(secondCheckpointId, ++timestamp); assertFlinkManifests(2); @@ -337,7 +341,8 @@ public void testRecoveryFromValidSnapshot() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -349,8 +354,8 @@ public void testRecoveryFromValidSnapshot() throws Exception { expectedRows.add(row); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row)); - harness.processElement(of(dataFile1), ++timestamp); - snapshot = harness.snapshot(++checkpointId, ++timestamp); + harness.processElement(of(++checkpointId, dataFile1), ++timestamp); + snapshot = harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -362,7 +367,8 @@ public void testRecoveryFromValidSnapshot() throws Exception { } // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.initializeState(snapshot); @@ -375,9 +381,9 @@ public void testRecoveryFromValidSnapshot() throws Exception { RowData row = SimpleDataUtil.createRowData(2, "world"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -400,7 +406,8 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except List expectedRows = Lists.newArrayList(); JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -411,15 +418,16 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(1, "hello"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-1", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - snapshot = harness.snapshot(++checkpointId, ++timestamp); + snapshot = harness.snapshot(checkpointId, ++timestamp); SimpleDataUtil.assertTableRows(table, ImmutableList.of(), branch); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); assertFlinkManifests(1); } - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.initializeState(snapshot); @@ -446,15 +454,15 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(2, "world"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - snapshot = harness.snapshot(++checkpointId, ++timestamp); + snapshot = harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); } // Redeploying flink job from external checkpoint. JobID newJobId = new JobID(); - try (OneInputStreamOperatorTestHarness harness = + try (OneInputStreamOperatorTestHarness harness = createStreamSink(newJobId)) { harness.setup(); harness.initializeState(snapshot); @@ -473,9 +481,9 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(3, "foo"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-3", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -496,7 +504,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { JobID oldJobId = new JobID(); OperatorID oldOperatorId; - try (OneInputStreamOperatorTestHarness harness = + try (OneInputStreamOperatorTestHarness harness = createStreamSink(oldJobId)) { harness.setup(); harness.open(); @@ -510,8 +518,8 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { tableRows.addAll(rows); DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); - harness.processElement(of(dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -528,7 +536,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { timestamp = 0; JobID newJobId = new JobID(); OperatorID newOperatorId; - try (OneInputStreamOperatorTestHarness harness = + try (OneInputStreamOperatorTestHarness harness = createStreamSink(newJobId)) { harness.setup(); harness.open(); @@ -542,8 +550,8 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { tableRows.addAll(rows); DataFile dataFile = writeDataFile("data-new-1", rows); - harness.processElement(of(dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -567,7 +575,8 @@ public void testMultipleJobsWriteSameTable() throws Exception { int checkpointId = i / 3; JobID jobId = jobs[jobIndex]; OperatorID operatorId = operatorIds[jobIndex]; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.open(); @@ -579,7 +588,7 @@ public void testMultipleJobsWriteSameTable() throws Exception { tableRows.addAll(rows); DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId + 1, dataFile), ++timestamp); harness.snapshot(checkpointId + 1, ++timestamp); assertFlinkManifests(1); @@ -603,8 +612,10 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { JobID jobId = new JobID(); OperatorID operatorId1 = new OperatorID(); OperatorID operatorId2 = new OperatorID(); - try (OneInputStreamOperatorTestHarness harness1 = createStreamSink(jobId); - OneInputStreamOperatorTestHarness harness2 = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness1 = + createStreamSink(jobId); + OneInputStreamOperatorTestHarness harness2 = + createStreamSink(jobId)) { harness1.getStreamConfig().setOperatorID(operatorId1); harness1.setup(); harness1.open(); @@ -620,14 +631,14 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { expectedRows.add(row1); DataFile dataFile1 = writeDataFile("data-1-1", ImmutableList.of(row1)); - harness1.processElement(of(dataFile1), ++timestamp); - snapshot1 = harness1.snapshot(++checkpointId, ++timestamp); + harness1.processElement(of(++checkpointId, dataFile1), ++timestamp); + snapshot1 = harness1.snapshot(checkpointId, ++timestamp); RowData row2 = SimpleDataUtil.createRowData(1, "hello2"); expectedRows.add(row2); DataFile dataFile2 = writeDataFile("data-1-2", ImmutableList.of(row2)); - harness2.processElement(of(dataFile2), ++timestamp); + harness2.processElement(of(checkpointId, dataFile2), ++timestamp); snapshot2 = harness2.snapshot(checkpointId, ++timestamp); assertFlinkManifests(2); @@ -643,8 +654,10 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { } // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness harness1 = createStreamSink(jobId); - OneInputStreamOperatorTestHarness harness2 = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness1 = + createStreamSink(jobId); + OneInputStreamOperatorTestHarness harness2 = + createStreamSink(jobId)) { harness1.getStreamConfig().setOperatorID(operatorId1); harness1.setup(); harness1.initializeState(snapshot1); @@ -668,13 +681,13 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { expectedRows.add(row1); DataFile dataFile1 = writeDataFile("data-2-1", ImmutableList.of(row1)); - harness1.processElement(of(dataFile1), ++timestamp); - harness1.snapshot(++checkpointId, ++timestamp); + harness1.processElement(of(++checkpointId, dataFile1), ++timestamp); + harness1.snapshot(checkpointId, ++timestamp); RowData row2 = SimpleDataUtil.createRowData(2, "world2"); expectedRows.add(row2); DataFile dataFile2 = writeDataFile("data-2-2", ImmutableList.of(row2)); - harness2.processElement(of(dataFile2), ++timestamp); + harness2.processElement(of(checkpointId, dataFile2), ++timestamp); harness2.snapshot(checkpointId, ++timestamp); assertFlinkManifests(2); @@ -694,7 +707,8 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { public void testBoundedStream() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -706,13 +720,14 @@ public void testBoundedStream() throws Exception { List tableRows = Lists.newArrayList(SimpleDataUtil.createRowData(1, "word-1")); DataFile dataFile = writeDataFile("data-1", tableRows); - harness.processElement(of(dataFile), 1); + harness.processElement(of(IcebergStreamWriter.END_INPUT_CHECKPOINT_ID, dataFile), 1); ((BoundedOneInput) harness.getOneInputOperator()).endInput(); assertFlinkManifests(0); SimpleDataUtil.assertTableRows(table, tableRows, branch); assertSnapshotSize(1); - assertMaxCommittedCheckpointId(jobId, operatorId, Long.MAX_VALUE); + assertMaxCommittedCheckpointId( + jobId, operatorId, IcebergStreamWriter.END_INPUT_CHECKPOINT_ID); assertThat(SimpleDataUtil.latestSnapshot(table, branch).summary()) .containsEntry("flink.test", TestIcebergFilesCommitter.class.getName()); } @@ -725,7 +740,8 @@ public void testFlinkManifests() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -735,7 +751,7 @@ public void testFlinkManifests() throws Exception { RowData row1 = SimpleDataUtil.createRowData(1, "hello"); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + harness.processElement(of(checkpoint, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 @@ -775,7 +791,8 @@ public void testDeleteFiles() throws Exception { OperatorID operatorId; FileAppenderFactory appenderFactory = createDeletableAppenderFactory(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -784,7 +801,7 @@ public void testDeleteFiles() throws Exception { RowData row1 = SimpleDataUtil.createInsert(1, "aaa"); DataFile dataFile1 = writeDataFile("data-file-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + harness.processElement(of(checkpoint, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 @@ -816,13 +833,15 @@ public void testDeleteFiles() throws Exception { RowData delete1 = SimpleDataUtil.createDelete(1, "aaa"); DeleteFile deleteFile1 = writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete1)); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); harness.processElement( - WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile1).build(), + new FlinkWriteResult( + ++checkpoint, + WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile1).build()), ++timestamp); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); // 5. snapshotState for checkpoint#2 - harness.snapshot(++checkpoint, ++timestamp); + harness.snapshot(checkpoint, ++timestamp); assertFlinkManifests(2); // 6. notifyCheckpointComplete for checkpoint#2 @@ -846,7 +865,8 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { OperatorID operatorId; FileAppenderFactory appenderFactory = createDeletableAppenderFactory(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -860,7 +880,9 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { DeleteFile deleteFile1 = writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete3)); harness.processElement( - WriteResult.builder().addDataFiles(dataFile1).addDeleteFiles(deleteFile1).build(), + new FlinkWriteResult( + checkpoint, + WriteResult.builder().addDataFiles(dataFile1).addDeleteFiles(deleteFile1).build()), ++timestamp); // The 1th snapshotState. @@ -872,11 +894,13 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { DeleteFile deleteFile2 = writeEqDeleteFile(appenderFactory, "delete-file-2", ImmutableList.of(delete2)); harness.processElement( - WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile2).build(), + new FlinkWriteResult( + ++checkpoint, + WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile2).build()), ++timestamp); // The 2nd snapshotState. - harness.snapshot(++checkpoint, ++timestamp); + harness.snapshot(checkpoint, ++timestamp); // Notify the 2nd snapshot to complete. harness.notifyOfCompletedCheckpoint(checkpoint); @@ -887,6 +911,79 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { } } + /** + * The testcase is to simulate upserting to an Iceberg V2 table, and facing the following + * scenario: + * + *
    + *
  • A specific row is updated + *
  • The prepareSnapshotPreBarrier triggered + *
  • Checkpoint failed for reasons outside of the Iceberg connector + *
  • The specific row is updated again in the second checkpoint as well + *
  • Second snapshot is triggered, and finished + *
+ * + *

Previously the files from the 2 snapshots were committed in a single Iceberg commit, as a + * results duplicate rows were created in the table. + * + * @throws Exception Exception + */ + @TestTemplate + public void testCommitMultipleCheckpointsForV2Table() throws Exception { + assumeThat(formatVersion) + .as("Only support equality-delete in format v2 or later.") + .isGreaterThan(1); + + long timestamp = 0; + long checkpoint = 10; + + JobID jobId = new JobID(); + OperatorID operatorId; + + FileAppenderFactory appenderFactory = + new FlinkAppenderFactory( + table, + table.schema(), + FlinkSchemaUtil.convert(table.schema()), + table.properties(), + table.spec(), + new int[] {table.schema().findField("id").fieldId()}, + table.schema(), + null); + + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + RowData insert1 = null; + RowData insert2 = null; + for (int i = 1; i <= 3; i++) { + insert1 = SimpleDataUtil.createInsert(1, "aaa" + i); + insert2 = SimpleDataUtil.createInsert(2, "bbb" + i); + DataFile dataFile = writeDataFile("data-file-" + i, ImmutableList.of(insert1, insert2)); + DeleteFile deleteFile = + writeEqDeleteFile( + appenderFactory, "delete-file-" + i, ImmutableList.of(insert1, insert2)); + harness.processElement( + new FlinkWriteResult( + ++checkpoint, + WriteResult.builder().addDataFiles(dataFile).addDeleteFiles(deleteFile).build()), + ++timestamp); + } + + harness.snapshot(checkpoint, ++timestamp); + harness.notifyOfCompletedCheckpoint(checkpoint); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(insert1, insert2), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); + assertFlinkManifests(0); + assertThat(table.snapshots()).hasSize(3); + } + } + @TestTemplate public void testSpecEvolution() throws Exception { long timestamp = 0; @@ -899,7 +996,8 @@ public void testSpecEvolution() throws Exception { DataFile dataFile; int specId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -910,7 +1008,7 @@ public void testSpecEvolution() throws Exception { RowData rowData = SimpleDataUtil.createRowData(checkpointId, "hello" + checkpointId); // table unpartitioned dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(rowData)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId, dataFile), ++timestamp); rows.add(rowData); harness.snapshot(checkpointId, ++timestamp); @@ -929,7 +1027,7 @@ public void testSpecEvolution() throws Exception { rowData = SimpleDataUtil.createRowData(checkpointId, "hello" + checkpointId); // write data with old partition spec dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(rowData), oldSpec, null); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId, dataFile), ++timestamp); rows.add(rowData); snapshot = harness.snapshot(checkpointId, ++timestamp); @@ -947,7 +1045,8 @@ public void testSpecEvolution() throws Exception { } // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.initializeState(snapshot); @@ -963,7 +1062,7 @@ public void testSpecEvolution() throws Exception { partition.set(0, checkpointId); dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(row), table.spec(), partition); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId, dataFile), ++timestamp); rows.add(row); harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); @@ -1079,8 +1178,8 @@ private DataFile writeDataFile( private void assertMaxCommittedCheckpointId(JobID jobID, OperatorID operatorID, long expectedId) { table.refresh(); long actualId = - IcebergFilesCommitter.getMaxCommittedCheckpointId( - table, jobID.toString(), operatorID.toHexString(), branch); + SinkUtil.getMaxCommittedCheckpointId( + table, jobID.toString(), operatorID.toString(), branch); assertThat(actualId).isEqualTo(expectedId); } @@ -1089,7 +1188,7 @@ private void assertSnapshotSize(int expectedSnapshotSize) { assertThat(table.snapshots()).hasSize(expectedSnapshotSize); } - private OneInputStreamOperatorTestHarness createStreamSink(JobID jobID) + private OneInputStreamOperatorTestHarness createStreamSink(JobID jobID) throws Exception { TestOperatorFactory factory = TestOperatorFactory.of(table.location(), branch, table.spec()); return new OneInputStreamOperatorTestHarness<>(factory, createEnvironment(jobID)); @@ -1109,7 +1208,7 @@ private static MockEnvironment createEnvironment(JobID jobID) { } private static class TestOperatorFactory extends AbstractStreamOperatorFactory - implements OneInputStreamOperatorFactory { + implements OneInputStreamOperatorFactory { private final String tablePath; private final String branch; private final PartitionSpec spec; diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSink.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSink.java new file mode 100644 index 000000000000..7f355c1e8403 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSink.java @@ -0,0 +1,436 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.sink.IcebergSink.Builder; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergSink extends TestFlinkIcebergSinkBase { + + private TableLoader tableLoader; + + @Parameter(index = 0) + private FileFormat format; + + @Parameter(index = 1) + private int parallelism; + + @Parameter(index = 2) + private boolean partitioned; + + @Parameters(name = "format={0}, parallelism={1}, partitioned={2}") + public static Object[][] parameters() { + return new Object[][] { + {FileFormat.AVRO, 1, true}, + {FileFormat.AVRO, 1, false}, + {FileFormat.AVRO, 2, true}, + {FileFormat.AVRO, 2, false}, + {FileFormat.ORC, 1, true}, + {FileFormat.ORC, 1, false}, + {FileFormat.ORC, 2, true}, + {FileFormat.ORC, 2, false}, + {FileFormat.PARQUET, 1, true}, + {FileFormat.PARQUET, 1, false}, + {FileFormat.PARQUET, 2, true}, + {FileFormat.PARQUET, 2, false} + }; + } + + @BeforeEach + void before() throws IOException { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + void testWriteRowData() throws Exception { + List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) + .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); + + IcebergSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream"); + + // Assert the iceberg table's records. + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } + + @TestTemplate + void testWriteRow() throws Exception { + testWriteRow(null, DistributionMode.NONE); + } + + @TestTemplate + void testWriteRowWithTableSchema() throws Exception { + testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); + } + + @TestTemplate + void testJobNoneDistributeMode() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + testWriteRow(null, DistributionMode.NONE); + + if (parallelism > 1) { + if (partitioned) { + int files = partitionFiles("aaa") + partitionFiles("bbb") + partitionFiles("ccc"); + assertThat(files).as("Should have more than 3 files in iceberg table.").isGreaterThan(3); + } + } + } + + @TestTemplate + void testJobHashDistributionMode() { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + assertThatThrownBy(() -> testWriteRow(null, DistributionMode.RANGE)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Flink does not support 'range' write distribution mode now."); + } + + @TestTemplate + void testJobNullDistributionMode() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + testWriteRow(null, null); + + if (partitioned) { + assertThat(partitionFiles("aaa")) + .as("There should be only 1 data file in partition 'aaa'") + .isEqualTo(1); + assertThat(partitionFiles("bbb")) + .as("There should be only 1 data file in partition 'bbb'") + .isEqualTo(1); + assertThat(partitionFiles("ccc")) + .as("There should be only 1 data file in partition 'ccc'") + .isEqualTo(1); + } + } + + @TestTemplate + void testPartitionWriteMode() throws Exception { + testWriteRow(null, DistributionMode.HASH); + if (partitioned) { + assertThat(partitionFiles("aaa")) + .as("There should be only 1 data file in partition 'aaa'") + .isEqualTo(1); + assertThat(partitionFiles("bbb")) + .as("There should be only 1 data file in partition 'bbb'") + .isEqualTo(1); + assertThat(partitionFiles("ccc")) + .as("There should be only 1 data file in partition 'ccc'") + .isEqualTo(1); + } + } + + @TestTemplate + void testShuffleByPartitionWithSchema() throws Exception { + testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.HASH); + if (partitioned) { + assertThat(partitionFiles("aaa")) + .as("There should be only 1 data file in partition 'aaa'") + .isEqualTo(1); + assertThat(partitionFiles("bbb")) + .as("There should be only 1 data file in partition 'bbb'") + .isEqualTo(1); + assertThat(partitionFiles("ccc")) + .as("There should be only 1 data file in partition 'ccc'") + .isEqualTo(1); + } + } + + @TestTemplate + void testTwoSinksInDisjointedDAG() throws Exception { + Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); + + Table leftTable = + CATALOG_EXTENSION + .catalog() + .createTable( + TableIdentifier.of("left"), + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + props); + TableLoader leftTableLoader = + TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("left")); + + Table rightTable = + CATALOG_EXTENSION + .catalog() + .createTable( + TableIdentifier.of("right"), + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + props); + TableLoader rightTableLoader = + TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("right")); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + env.getConfig().disableAutoGeneratedUIDs(); + + List leftRows = createRows("left-"); + DataStream leftStream = + env.fromCollection(leftRows, ROW_TYPE_INFO) + .name("leftCustomSource") + .uid("leftCustomSource"); + IcebergSink.forRow(leftStream, SimpleDataUtil.FLINK_SCHEMA) + .table(leftTable) + .tableLoader(leftTableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .distributionMode(DistributionMode.NONE) + .uidSuffix("leftIcebergSink") + .append(); + + List rightRows = createRows("right-"); + DataStream rightStream = + env.fromCollection(rightRows, ROW_TYPE_INFO) + .name("rightCustomSource") + .uid("rightCustomSource"); + IcebergSink.forRow(rightStream, SimpleDataUtil.FLINK_SCHEMA) + .table(rightTable) + .tableLoader(rightTableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .distributionMode(DistributionMode.HASH) + .uidSuffix("rightIcebergSink") + .setSnapshotProperty("flink.test", TestIcebergSink.class.getName()) + .snapshotProperties(Collections.singletonMap("direction", "rightTable")) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(leftTable, convertToRowData(leftRows)); + SimpleDataUtil.assertTableRows(rightTable, convertToRowData(rightRows)); + + leftTable.refresh(); + + assertThat(leftTable.currentSnapshot().summary().get("flink.test")).isNull(); + assertThat(leftTable.currentSnapshot().summary().get("direction")).isNull(); + + assertThat(rightTable.currentSnapshot().summary().get("flink.test")) + .isEqualTo(TestIcebergSink.class.getName()); + assertThat(rightTable.currentSnapshot().summary().get("direction")).isEqualTo("rightTable"); + } + + @TestTemplate + void testOverrideWriteConfigWithUnknownDistributionMode() { + Map newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), "UNRECOGNIZED"); + + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .setAll(newProps) + .append(); + + assertThatThrownBy(() -> env.execute("Test Iceberg DataStream")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid distribution mode: UNRECOGNIZED"); + } + + @TestTemplate + void testOverrideWriteConfigWithUnknownFileFormat() { + Map newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.WRITE_FORMAT.key(), "UNRECOGNIZED"); + + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + Builder builder = + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .setAll(newProps) + .uidSuffix("ingestion"); + assertThatThrownBy(builder::append) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid file format: UNRECOGNIZED"); + } + + @TestTemplate + void testWriteRowWithTableRefreshInterval() throws Exception { + List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) + .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); + + Configuration flinkConf = new Configuration(); + flinkConf.setString(FlinkWriteOptions.TABLE_REFRESH_INTERVAL.key(), "100ms"); + + IcebergSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .flinkConf(flinkConf) + .writeParallelism(parallelism) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream"); + + // Assert the iceberg table's records. + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } + + @TestTemplate + void testOperatorsUidNameNoUidSuffix() throws Exception { + List rows = createRows(""); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO).uid("mySourceId"); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .distributionMode(DistributionMode.HASH) + .append(); + + Transformation firstTransformation = env.getTransformations().get(0); + Transformation secondTransformation = env.getTransformations().get(1); + assertThat(firstTransformation.getUid()).isEqualTo("Sink pre-writer mapper: hadoop.default.t"); + assertThat(firstTransformation.getName()).isEqualTo("Sink pre-writer mapper: hadoop.default.t"); + assertThat(secondTransformation.getUid()).isEqualTo("hadoop.default.t"); + assertThat(secondTransformation.getName()).isEqualTo("hadoop.default.t"); + } + + @TestTemplate + void testOperatorsUidNameWitUidSuffix() throws Exception { + List rows = createRows(""); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO).uid("mySourceId"); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .distributionMode(DistributionMode.HASH) + .uidSuffix("data-ingestion") + .append(); + + Transformation firstTransformation = env.getTransformations().get(0); + Transformation secondTransformation = env.getTransformations().get(1); + assertThat(firstTransformation.getUid()).isEqualTo("Sink pre-writer mapper: data-ingestion"); + assertThat(firstTransformation.getName()).isEqualTo("Sink pre-writer mapper: data-ingestion"); + assertThat(secondTransformation.getUid()).isEqualTo("data-ingestion"); + assertThat(secondTransformation.getName()).isEqualTo("data-ingestion"); + } + + private void testWriteRow(TableSchema tableSchema, DistributionMode distributionMode) + throws Exception { + List rows = createRows(""); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO).uid("mySourceId"); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .writeParallelism(parallelism) + .distributionMode(distributionMode) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java new file mode 100644 index 000000000000..a8c8892af11c --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.util.List; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergSinkBranch extends TestFlinkIcebergSinkBase { + + @Parameter(index = 0) + private String branch; + + @Parameters(name = "branch = {0}") + public static Object[][] parameters() { + return new Object[][] {new Object[] {"main"}, new Object[] {"testBranch"}}; + } + + @BeforeEach + public void before() throws IOException { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + FileFormat.AVRO.name(), + TableProperties.FORMAT_VERSION, + "1")); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100); + + tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testWriteRowWithTableSchema() throws Exception { + testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); + verifyOtherBranchUnmodified(); + } + + private void testWriteRow(TableSchema tableSchema, DistributionMode distributionMode) + throws Exception { + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .toBranch(branch) + .distributionMode(distributionMode) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(table, convertToRowData(rows), branch); + SimpleDataUtil.assertTableRows( + table, + ImmutableList.of(), + branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH); + + verifyOtherBranchUnmodified(); + } + + private void verifyOtherBranchUnmodified() { + String otherBranch = + branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; + if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { + assertThat(table.currentSnapshot()).isNull(); + } + + assertThat(table.snapshot(otherBranch)).isNull(); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java new file mode 100644 index 000000000000..007d5f314946 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java @@ -0,0 +1,267 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.apache.iceberg.flink.TestFixtures.DATABASE; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.util.List; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.BoundedTestSource; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; + +@ExtendWith(ParameterizedTestExtension.class) +@Timeout(value = 60) +public class TestIcebergSinkV2 extends TestFlinkIcebergSinkV2Base { + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); + + @BeforeEach + public void setupTable() { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + format.name(), + TableProperties.FORMAT_VERSION, + String.valueOf(FORMAT_V2))); + + table + .updateProperties() + .set(TableProperties.DEFAULT_FILE_FORMAT, format.name()) + .set(TableProperties.WRITE_DISTRIBUTION_MODE, writeDistributionMode) + .commit(); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100L) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testCheckAndGetEqualityFieldIds() { + table + .updateSchema() + .allowIncompatibleChanges() + .addRequiredColumn("type", Types.StringType.get()) + .setIdentifierFields("type") + .commit(); + + DataStream dataStream = + env.addSource(new BoundedTestSource<>(ImmutableList.of()), ROW_TYPE_INFO); + IcebergSink.Builder builder = + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA).table(table); + + // Use user-provided equality field column as equality field id list + builder.equalityFieldColumns(Lists.newArrayList("id")); + assertThat(SinkUtil.checkAndGetEqualityFieldIds(table, Lists.newArrayList("id"))) + .containsExactlyInAnyOrder(table.schema().findField("id").fieldId()); + } + + @TestTemplate + public void testChangeLogOnIdKey() throws Exception { + testChangeLogOnIdKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertOnlyDeletesOnDataKey() throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of(row("+I", 1, "aaa")), + ImmutableList.of(row("-D", 1, "aaa"), row("-D", 2, "bbb"))); + + List> expectedRecords = + ImmutableList.of(ImmutableList.of(record(1, "aaa")), ImmutableList.of()); + + testChangeLogs( + ImmutableList.of("data"), + row -> row.getField(ROW_DATA_POS), + true, + elementsPerCheckpoint, + expectedRecords, + SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testChangeLogOnDataKey() throws Exception { + testChangeLogOnDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testChangeLogOnIdDataKey() throws Exception { + testChangeLogOnIdDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testChangeLogOnSameKey() throws Exception { + testChangeLogOnSameKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertModeCheck() throws Exception { + DataStream dataStream = + env.addSource(new BoundedTestSource<>(ImmutableList.of()), ROW_TYPE_INFO); + IcebergSink.Builder builder = + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .upsert(true); + + assertThatThrownBy( + () -> + builder + .equalityFieldColumns(ImmutableList.of("id", "data")) + .overwrite(true) + .append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); + + assertThatThrownBy( + () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + } + + @TestTemplate + public void testUpsertOnIdKey() throws Exception { + testUpsertOnIdKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertOnDataKey() throws Exception { + testUpsertOnDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertOnIdDataKey() throws Exception { + testUpsertOnIdDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testDeleteStats() throws Exception { + assumeThat(format).isNotEqualTo(FileFormat.AVRO); + + List> elementsPerCheckpoint = + ImmutableList.of( + // Checkpoint #1 + ImmutableList.of(row("+I", 1, "aaa"), row("-D", 1, "aaa"), row("+I", 1, "aaa"))); + + List> expectedRecords = ImmutableList.of(ImmutableList.of(record(1, "aaa"))); + + testChangeLogs( + ImmutableList.of("id", "data"), + row -> Row.of(row.getField(ROW_ID_POS), row.getField(ROW_DATA_POS)), + false, + elementsPerCheckpoint, + expectedRecords, + "main"); + + DeleteFile deleteFile = table.currentSnapshot().addedDeleteFiles(table.io()).iterator().next(); + String fromStat = + new String( + deleteFile.lowerBounds().get(MetadataColumns.DELETE_FILE_PATH.fieldId()).array()); + DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + assumeThat(fromStat).isEqualTo(dataFile.path().toString()); + } + + protected void testChangeLogs( + List equalityFieldColumns, + KeySelector keySelector, + boolean insertAsUpsert, + List> elementsPerCheckpoint, + List> expectedRecordsPerCheckpoint, + String branch) + throws Exception { + DataStream dataStream = + env.addSource(new BoundedTestSource<>(elementsPerCheckpoint), ROW_TYPE_INFO); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .equalityFieldColumns(equalityFieldColumns) + .upsert(insertAsUpsert) + .toBranch(branch) + .uidSuffix("sink") + .append(); + + // Execute the program. + env.execute("Test Iceberg Change-Log DataStream."); + + table.refresh(); + List snapshots = findValidSnapshots(); + int expectedSnapshotNum = expectedRecordsPerCheckpoint.size(); + assertThat(snapshots).hasSize(expectedSnapshotNum); + + for (int i = 0; i < expectedSnapshotNum; i++) { + long snapshotId = snapshots.get(i).snapshotId(); + List expectedRecords = expectedRecordsPerCheckpoint.get(i); + assertThat(actualRowSet(snapshotId, "*")) + .as("Should have the expected records for the checkpoint#" + i) + .isEqualTo(expectedRowSet(expectedRecords.toArray(new Record[0]))); + } + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2Branch.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2Branch.java new file mode 100644 index 000000000000..4896f7f48c17 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2Branch.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Branch { + + @BeforeEach + @Override + public void before() throws IOException { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + FileFormat.AVRO.name(), + TableProperties.FORMAT_VERSION, + "2")); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100); + + tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testChangeLogOnIdKey() throws Exception { + testChangeLogOnIdKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testChangeLogOnDataKey() throws Exception { + testChangeLogOnDataKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testChangeLogOnIdDataKey() throws Exception { + testChangeLogOnIdDataKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testUpsertOnIdKey() throws Exception { + testUpsertOnIdKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testUpsertOnDataKey() throws Exception { + testUpsertOnDataKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testUpsertOnIdDataKey() throws Exception { + testUpsertOnIdDataKey(branch); + verifyOtherBranchUnmodified(); + } + + private void verifyOtherBranchUnmodified() { + String otherBranch = + branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; + if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { + assertThat(table.currentSnapshot()); + } + + assertThat(table.snapshot(otherBranch)).isNull(); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java index 50283f7ad215..e13721a9f170 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java @@ -28,6 +28,7 @@ import java.util.Locale; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; import org.apache.flink.streaming.api.operators.BoundedOneInput; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.table.api.DataTypes; @@ -102,7 +103,7 @@ public void before() throws IOException { @TestTemplate public void testWritingTable() throws Exception { long checkpointId = 1L; - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { // The first checkpoint testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); @@ -111,7 +112,8 @@ public void testWritingTable() throws Exception { testHarness.prepareSnapshotPreBarrier(checkpointId); int expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); @@ -123,7 +125,8 @@ public void testWritingTable() throws Exception { testHarness.prepareSnapshotPreBarrier(checkpointId); expectedDataFiles = partitioned ? 4 : 2; - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); @@ -148,14 +151,15 @@ public void testWritingTable() throws Exception { public void testSnapshotTwice() throws Exception { long checkpointId = 1; long timestamp = 1; - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), timestamp++); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), timestamp); testHarness.prepareSnapshotPreBarrier(checkpointId++); int expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); @@ -163,7 +167,10 @@ public void testSnapshotTwice() throws Exception { for (int i = 0; i < 5; i++) { testHarness.prepareSnapshotPreBarrier(checkpointId++); - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + result = + WriteResult.builder() + .addAll(getWriteResults(testHarness.extractOutputValues())) + .build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); } @@ -172,14 +179,14 @@ public void testSnapshotTwice() throws Exception { @TestTemplate public void testTableWithoutSnapshot() throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { assertThat(testHarness.extractOutputValues()).isEmpty(); } // Even if we closed the iceberg stream writer, there's no orphan data file. assertThat(scanDataFiles()).isEmpty(); - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); // Still not emit the data file yet, because there is no checkpoint. @@ -212,7 +219,7 @@ private Set scanDataFiles() throws IOException { @TestTemplate public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); @@ -221,13 +228,15 @@ public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); int expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); // Datafiles should not be sent again assertThat(result.dataFiles()).hasSize(expectedDataFiles); @@ -236,7 +245,7 @@ public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { @TestTemplate public void testBoundedStreamTriggeredEndInputBeforeTriggeringCheckpoint() throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); @@ -244,13 +253,15 @@ public void testBoundedStreamTriggeredEndInputBeforeTriggeringCheckpoint() throw testHarness.endInput(); int expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); testHarness.prepareSnapshotPreBarrier(1L); - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); // It should be ensured that after endInput is triggered, when prepareSnapshotPreBarrier // is triggered, write should only send WriteResult once @@ -275,7 +286,7 @@ public void testTableWithTargetFileSize() throws Exception { } } - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { for (RowData row : rows) { testHarness.processElement(row, 1); @@ -283,7 +294,8 @@ public void testTableWithTargetFileSize() throws Exception { // snapshot the operator. testHarness.prepareSnapshotPreBarrier(1); - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(8); @@ -346,13 +358,14 @@ public void testPromotedFlinkDataType() throws Exception { record.copy(ImmutableMap.of("tinyint", 2, "smallint", 0, "int", 102)), record.copy(ImmutableMap.of("tinyint", 3, "smallint", 32767, "int", 103))); - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter(icebergTable, flinkSchema)) { for (RowData row : rows) { testHarness.processElement(row, 1); } testHarness.prepareSnapshotPreBarrier(1); - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(partitioned ? 3 : 1); @@ -365,12 +378,18 @@ public void testPromotedFlinkDataType() throws Exception { SimpleDataUtil.assertTableRecords(location, expected); } - private OneInputStreamOperatorTestHarness createIcebergStreamWriter() + private static List getWriteResults(List flinkWriteResults) { + return flinkWriteResults.stream() + .map(FlinkWriteResult::writeResult) + .collect(Collectors.toList()); + } + + private OneInputStreamOperatorTestHarness createIcebergStreamWriter() throws Exception { return createIcebergStreamWriter(table, SimpleDataUtil.FLINK_SCHEMA); } - private OneInputStreamOperatorTestHarness createIcebergStreamWriter( + private OneInputStreamOperatorTestHarness createIcebergStreamWriter( Table icebergTable, TableSchema flinkSchema) throws Exception { RowType flinkRowType = FlinkSink.toFlinkRowType(icebergTable.schema(), flinkSchema); FlinkWriteConf flinkWriteConfig = @@ -379,7 +398,7 @@ private OneInputStreamOperatorTestHarness createIcebergStr IcebergStreamWriter streamWriter = FlinkSink.createStreamWriter(() -> icebergTable, flinkWriteConfig, flinkRowType, null); - OneInputStreamOperatorTestHarness harness = + OneInputStreamOperatorTestHarness harness = new OneInputStreamOperatorTestHarness<>(streamWriter, 1, 1, 0); harness.setup(); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java index d5a0bebc74e7..c95a9955b104 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java @@ -380,7 +380,9 @@ private static Map>> runPartitioner( return partitionResults; } - /** @param expectedAssignmentInfo excluding closing cost */ + /** + * @param expectedAssignmentInfo excluding closing cost + */ private void validatePartitionResults( Map> expectedAssignmentInfo, Map>> partitionResults, diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java index e1162c3225b1..2dc5bc5c658e 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java @@ -33,10 +33,12 @@ import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; /** Use the FlinkSource */ public class TestFlinkSourceSql extends TestSqlBase { + @BeforeEach @Override public void before() throws IOException { SqlHelpers.sql( diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java index b7447d15c05a..db8647f054ae 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java @@ -24,8 +24,6 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -130,11 +128,8 @@ protected List run( sourceBuilder.properties(options); DataStream stream = - env.fromSource( - sourceBuilder.build(), - WatermarkStrategy.noWatermarks(), - "testBasicRead", - TypeInformation.of(RowData.class)) + sourceBuilder + .buildStream(env) .map( new RowDataToRowMapper( FlinkSchemaUtil.convert( diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java index 7bfed00a9eb4..4e649d15b1ce 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java @@ -52,6 +52,7 @@ import org.apache.iceberg.flink.data.RowDataToRowMapper; import org.apache.iceberg.flink.sink.AvroGenericRecordToRowDataMapper; import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; +import org.apache.iceberg.flink.source.reader.AvroGenericRecordConverter; import org.apache.iceberg.flink.source.reader.AvroGenericRecordReaderFunction; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; @@ -69,12 +70,13 @@ public class TestIcebergSourceBoundedGenericRecord { private static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); - @Parameters(name = "format={0}, parallelism = {1}") + @Parameters(name = "format={0}, parallelism = {1}, useConverter = {2}") public static Object[][] parameters() { return new Object[][] { - {FileFormat.AVRO, 2}, - {FileFormat.PARQUET, 2}, - {FileFormat.ORC, 2} + {FileFormat.AVRO, 2, true}, + {FileFormat.PARQUET, 2, true}, + {FileFormat.PARQUET, 2, false}, + {FileFormat.ORC, 2, true} }; } @@ -84,6 +86,9 @@ public static Object[][] parameters() { @Parameter(index = 1) private int parallelism; + @Parameter(index = 2) + private boolean useConverter; + @TestTemplate public void testUnpartitionedTable() throws Exception { Table table = @@ -147,24 +152,15 @@ private List run( table = tableLoader.loadTable(); } - AvroGenericRecordReaderFunction readerFunction = - new AvroGenericRecordReaderFunction( - TestFixtures.TABLE_IDENTIFIER.name(), - new Configuration(), - table.schema(), - null, - null, - false, - table.io(), - table.encryption(), - filters); + Schema readSchema = projectedSchema != null ? projectedSchema : table.schema(); + IcebergSource.Builder sourceBuilder; + if (useConverter) { + sourceBuilder = createSourceBuilderWithConverter(table, readSchema, config); + } else { + sourceBuilder = + createSourceBuilderWithReaderFunction(table, projectedSchema, filters, config); + } - IcebergSource.Builder sourceBuilder = - IcebergSource.builder() - .tableLoader(CATALOG_EXTENSION.tableLoader()) - .readerFunction(readerFunction) - .assignerFactory(new SimpleSplitAssignerFactory()) - .flinkConfig(config); if (projectedSchema != null) { sourceBuilder.project(projectedSchema); } @@ -172,7 +168,6 @@ private List run( sourceBuilder.filters(filters); sourceBuilder.setAll(options); - Schema readSchema = projectedSchema != null ? projectedSchema : table.schema(); RowType rowType = FlinkSchemaUtil.convert(readSchema); org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(readSchema, TestFixtures.TABLE_IDENTIFIER.name()); @@ -193,4 +188,35 @@ private List run( return Lists.newArrayList(iter); } } + + private IcebergSource.Builder createSourceBuilderWithReaderFunction( + Table table, Schema projected, List filters, Configuration config) { + AvroGenericRecordReaderFunction readerFunction = + new AvroGenericRecordReaderFunction( + TestFixtures.TABLE_IDENTIFIER.name(), + new Configuration(), + table.schema(), + projected, + null, + false, + table.io(), + table.encryption(), + filters); + + return IcebergSource.builder() + .tableLoader(CATALOG_EXTENSION.tableLoader()) + .readerFunction(readerFunction) + .assignerFactory(new SimpleSplitAssignerFactory()) + .flinkConfig(config); + } + + private IcebergSource.Builder createSourceBuilderWithConverter( + Table table, Schema readSchema, Configuration config) { + AvroGenericRecordConverter converter = + AvroGenericRecordConverter.fromIcebergSchema(readSchema, table.name()); + return IcebergSource.forOutputType(converter) + .tableLoader(CATALOG_EXTENSION.tableLoader()) + .assignerFactory(new SimpleSplitAssignerFactory()) + .flinkConfig(config); + } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java index 0f41c5af4c95..d3713e296014 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java @@ -37,7 +37,7 @@ public class TestIcebergSourceBoundedSql extends TestIcebergSourceBounded { @BeforeEach public void before() throws IOException { Configuration tableConf = getTableEnv().getConfig().getConfiguration(); - tableConf.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), true); + tableConf.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); SqlHelpers.sql( getTableEnv(), "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java new file mode 100644 index 000000000000..2908cb927269 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java @@ -0,0 +1,181 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source; + +import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.nio.file.Path; +import java.util.List; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; +import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.testutils.InternalMiniClusterExtension; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.data.RowDataToRowMapper; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +public class TestIcebergSourceInferParallelism { + private static final int NUM_TMS = 2; + private static final int SLOTS_PER_TM = 2; + private static final int PARALLELISM = NUM_TMS * SLOTS_PER_TM; + private static final int MAX_INFERRED_PARALLELISM = 3; + + @RegisterExtension + private static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(NUM_TMS) + .setNumberSlotsPerTaskManager(SLOTS_PER_TM) + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) + .build()); + + @RegisterExtension + protected static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + @TempDir private Path tmpDir; + + private Table table; + private GenericAppenderHelper dataAppender; + + @BeforeEach + public void before() throws IOException { + this.table = + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + this.dataAppender = new GenericAppenderHelper(table, FileFormat.PARQUET, tmpDir); + } + + @AfterEach + public void after() { + CATALOG_EXTENSION.catalog().dropTable(TestFixtures.TABLE_IDENTIFIER); + } + + @Test + public void testEmptyTable() throws Exception { + // Inferred parallelism should be at least 1 even if table is empty + test(1, 0); + } + + @Test + public void testTableWithFilesLessThanMaxInferredParallelism() throws Exception { + // Append files to the table + for (int i = 0; i < 2; ++i) { + List batch = RandomGenericData.generate(table.schema(), 1, 0); + dataAppender.appendToTable(batch); + } + + // Inferred parallelism should equal to 2 splits + test(2, 2); + } + + @Test + public void testTableWithFilesMoreThanMaxInferredParallelism() throws Exception { + // Append files to the table + for (int i = 0; i < MAX_INFERRED_PARALLELISM + 1; ++i) { + List batch = RandomGenericData.generate(table.schema(), 1, 0); + dataAppender.appendToTable(batch); + } + + // Inferred parallelism should be capped by the MAX_INFERRED_PARALLELISM + test(MAX_INFERRED_PARALLELISM, MAX_INFERRED_PARALLELISM + 1); + } + + private void test(int expectedParallelism, int expectedRecords) throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(PARALLELISM); + + Configuration config = new Configuration(); + config.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, true); + config.set( + FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX, + MAX_INFERRED_PARALLELISM); + + DataStream dataStream = + IcebergSource.forRowData() + .tableLoader(CATALOG_EXTENSION.tableLoader()) + .table(table) + .flinkConfig(config) + // force one file per split + .splitSize(1L) + .buildStream(env) + .map(new RowDataToRowMapper(FlinkSchemaUtil.convert(table.schema()))); + + DataStream.Collector collector = new DataStream.Collector<>(); + dataStream.collectAsync(collector); + JobClient jobClient = env.executeAsync(); + try (CloseableIterator iterator = collector.getOutput()) { + List result = Lists.newArrayList(); + while (iterator.hasNext()) { + result.add(iterator.next()); + } + + assertThat(result).hasSize(expectedRecords); + verifySourceParallelism( + expectedParallelism, miniCluster().getExecutionGraph(jobClient.getJobID()).get()); + } + } + + /** + * Borrowed this approach from Flink {@code FileSourceTextLinesITCase} to get source parallelism + * from execution graph. + */ + private static void verifySourceParallelism( + int expectedParallelism, AccessExecutionGraph executionGraph) { + AccessExecutionJobVertex sourceVertex = + executionGraph.getVerticesTopologically().iterator().next(); + assertThat(sourceVertex.getParallelism()).isEqualTo(expectedParallelism); + } + + /** + * Use reflection to get {@code InternalMiniClusterExtension} and {@code MiniCluster} to get + * execution graph and source parallelism. Haven't find other way via public APIS. + */ + private static MiniCluster miniCluster() throws Exception { + Field privateField = + MiniClusterExtension.class.getDeclaredField("internalMiniClusterExtension"); + privateField.setAccessible(true); + InternalMiniClusterExtension internalExtension = + (InternalMiniClusterExtension) privateField.get(MINI_CLUSTER_EXTENSION); + return internalExtension.getMiniCluster(); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java index 75f0a785a8c5..66bdeee1d407 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java @@ -40,6 +40,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; /** Use the IcebergSource (FLIP-27) */ @@ -49,11 +50,17 @@ public class TestIcebergSourceSql extends TestSqlBase { required(1, "t1", Types.TimestampType.withoutZone()), required(2, "t2", Types.LongType.get())); + @BeforeEach @Override public void before() throws IOException { TableEnvironment tableEnvironment = getTableEnv(); Configuration tableConf = tableEnvironment.getConfig().getConfiguration(); - tableConf.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), true); + tableConf.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); + // Disable inferring parallelism to avoid interfering watermark tests + // that check split assignment is ordered by the watermark column. + // The tests assumes default parallelism of 1 with single reader task + // in order to check the order of read records. + tableConf.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); tableEnvironment.getConfig().set("table.exec.resource.default-parallelism", "1"); SqlHelpers.sql( diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java index 992b712d9d69..f0d083060c1d 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java @@ -27,6 +27,7 @@ import java.util.List; import java.util.stream.Collectors; import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.common.TaskInfo; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.configuration.BatchExecutionOptions; @@ -48,8 +49,14 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.extension.RegisterExtension; +/** + * There is a infinite sleep in the test. Add a timeout to the test to avoid stuck situation in case + * anything goes wrong unexpectedly. + */ +@Timeout(value = 60) public class TestIcebergSpeculativeExecutionSupport extends TestBase { private static final int NUM_TASK_MANAGERS = 1; private static final int NUM_TASK_SLOTS = 3; @@ -144,9 +151,9 @@ public void testSpeculativeExecution() throws Exception { private static class TestingMap extends RichMapFunction { @Override public Row map(Row row) throws Exception { - // Put the subtasks with the first attempt to sleep to trigger speculative - // execution - if (getRuntimeContext().getTaskInfo().getAttemptNumber() <= 0) { + // Simulate slow subtask 0 with attempt 0 + TaskInfo taskInfo = getRuntimeContext().getTaskInfo(); + if (taskInfo.getIndexOfThisSubtask() == 0 && taskInfo.getAttemptNumber() <= 0) { Thread.sleep(Integer.MAX_VALUE); } @@ -169,6 +176,7 @@ private static Configuration configure() { // Use FLIP-27 source configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); + configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); // for speculative execution configuration.set(BatchExecutionOptions.SPECULATIVE_ENABLED, true); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java index 9cf953342a18..9cdcb72c12cf 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java @@ -217,27 +217,27 @@ public void testPrimitiveColumns() throws Exception { Row binaryCol = Row.of( - 52L, + 55L, 4L, 2L, null, Base64.getDecoder().decode("1111"), Base64.getDecoder().decode("2222")); - Row booleanCol = Row.of(32L, 4L, 0L, null, false, true); - Row decimalCol = Row.of(85L, 4L, 1L, null, new BigDecimal("1.00"), new BigDecimal("2.00")); - Row doubleCol = Row.of(85L, 4L, 0L, 1L, 1.0D, 2.0D); + Row booleanCol = Row.of(36L, 4L, 0L, null, false, true); + Row decimalCol = Row.of(91L, 4L, 1L, null, new BigDecimal("1.00"), new BigDecimal("2.00")); + Row doubleCol = Row.of(91L, 4L, 0L, 1L, 1.0D, 2.0D); Row fixedCol = Row.of( - 44L, + 47L, 4L, 2L, null, Base64.getDecoder().decode("1111"), Base64.getDecoder().decode("2222")); - Row floatCol = Row.of(71L, 4L, 0L, 2L, 0f, 0f); - Row intCol = Row.of(71L, 4L, 0L, null, 1, 2); - Row longCol = Row.of(79L, 4L, 0L, null, 1L, 2L); - Row stringCol = Row.of(79L, 4L, 0L, null, "1", "2"); + Row floatCol = Row.of(77L, 4L, 0L, 2L, 0f, 0f); + Row intCol = Row.of(77L, 4L, 0L, null, 1, 2); + Row longCol = Row.of(85L, 4L, 0L, null, 1L, 2L); + Row stringCol = Row.of(85L, 4L, 0L, null, "1", "2"); List expected = Lists.newArrayList( @@ -289,8 +289,8 @@ public void testSelectNestedValues() throws Exception { public void testNestedValues() throws Exception { createNestedTable(); - Row leafDoubleCol = Row.of(46L, 3L, 1L, 1L, 0.0D, 0.0D); - Row leafLongCol = Row.of(54L, 3L, 1L, null, 0L, 1L); + Row leafDoubleCol = Row.of(50L, 3L, 1L, 1L, 0.0D, 0.0D); + Row leafLongCol = Row.of(57L, 3L, 1L, null, 0L, 1L); Row metrics = Row.of(Row.of(leafDoubleCol, leafLongCol)); TestHelpers.assertRows( diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java index 0690b456e033..5767fa02c822 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java @@ -80,7 +80,9 @@ private void appendTwoSnapshots() throws IOException { snapshot2 = TABLE_RESOURCE.table().currentSnapshot(); } - /** @return the last enumerated snapshot id */ + /** + * @return the last enumerated snapshot id + */ private CycleResult verifyOneCycle( ContinuousSplitPlannerImpl splitPlanner, IcebergEnumeratorPosition lastPosition) throws Exception { diff --git a/flink/v1.20/flink/src/test/resources/META-INF/services/org.apache.flink.metrics.reporter.MetricReporterFactory b/flink/v1.20/flink/src/test/resources/META-INF/services/org.apache.flink.metrics.reporter.MetricReporterFactory new file mode 100644 index 000000000000..952255a52b7c --- /dev/null +++ b/flink/v1.20/flink/src/test/resources/META-INF/services/org.apache.flink.metrics.reporter.MetricReporterFactory @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +org.apache.iceberg.flink.maintenance.operator.MetricsReporterFactoryForTests diff --git a/format/puffin-spec.md b/format/puffin-spec.md index 1fe9e5b9bc56..7b4e3e6d96b3 100644 --- a/format/puffin-spec.md +++ b/format/puffin-spec.md @@ -121,7 +121,9 @@ distinct values converted to bytes using Iceberg's single-value serialization. The blob metadata for this blob may include following properties: -- `ndv`: estimate of number of distinct values, derived from the sketch. +- `ndv`: estimate of number of distinct values, derived from the sketch, + stored as non-negative integer value represented using decimal digits + with no leading or trailing spaces. ### Compression codecs diff --git a/format/spec.md b/format/spec.md index c322f8174fe2..6b80e876ed43 100644 --- a/format/spec.md +++ b/format/spec.md @@ -30,13 +30,13 @@ Versions 1 and 2 of the Iceberg spec are complete and adopted by the community. The format version number is incremented when new features are added that will break forward-compatibility---that is, when older readers would not read newer table features correctly. Tables may continue to be written with an older version of the spec to ensure compatibility by not using features that are not yet implemented by processing engines. -#### Version 1: Analytic Data Tables +### Version 1: Analytic Data Tables Version 1 of the Iceberg spec defines how to manage large analytic tables using immutable file formats: Parquet, Avro, and ORC. All version 1 data and metadata files are valid after upgrading a table to version 2. [Appendix E](#version-2) documents how to default version 2 fields when reading version 1 metadata. -#### Version 2: Row-level Deletes +### Version 2: Row-level Deletes Version 2 of the Iceberg spec adds row-level updates and deletes for analytic tables with immutable files. @@ -44,6 +44,14 @@ The primary change in version 2 adds delete files to encode rows that are delete In addition to row-level deletes, version 2 makes some requirements stricter for writers. The full set of changes are listed in [Appendix E](#version-2). +### Version 3: Extended Types and Capabilities + +Version 3 of the Iceberg spec extends data types and existing metadata structures to add new capabilities: + +* New data types: nanosecond timestamp(tz), unknown +* Default value support for columns +* Multi-argument transforms for partitioning and sorting +* Row Lineage tracking ## Goals @@ -67,7 +75,7 @@ Data files in snapshots are tracked by one or more manifest files that contain a The manifests that make up a snapshot are stored in a manifest list file. Each manifest list stores metadata about manifests, including partition stats and data file counts. These stats are used to avoid reading manifests that are not required for an operation. -#### Optimistic Concurrency +### Optimistic Concurrency An atomic swap of one table metadata file for another provides the basis for serializable isolation. Readers use the snapshot that was current when they load the table metadata and are not affected by changes until they refresh and pick up a new metadata location. @@ -77,7 +85,7 @@ If the snapshot on which an update is based is no longer current, the writer mus The conditions required by a write to successfully commit determines the isolation level. Writers can select what to validate and can make different isolation guarantees. -#### Sequence Numbers +### Sequence Numbers The relative age of data and delete files relies on a sequence number that is assigned to every successful commit. When a snapshot is created for a commit, it is optimistically assigned the next sequence number, and it is written into the snapshot's metadata. If the commit fails and must be retried, the sequence number is reassigned and written into new snapshot metadata. @@ -86,7 +94,7 @@ All manifests, data files, and delete files created for a snapshot inherit the s Inheriting the sequence number from manifest metadata allows writing a new manifest once and reusing it in commit retries. To change a sequence number for a retry, only the manifest list must be rewritten -- which would be rewritten anyway with the latest set of manifests. -#### Row-level Deletes +### Row-level Deletes Row-level deletes are stored in delete files. @@ -98,7 +106,7 @@ There are two ways to encode a row-level delete: Like data files, delete files are tracked by partition. In general, a delete file must be applied to older data files with the same partition; see [Scan Planning](#scan-planning) for details. Column metrics can be used to determine whether a delete file's rows overlap the contents of a data file or a scan range. -#### File System Operations +### File System Operations Iceberg only requires that file systems support the following operations: @@ -127,7 +135,7 @@ Tables do not require rename, except for tables that use atomic rename to implem #### Writer requirements -Some tables in this spec have columns that specify requirements for v1 and v2 tables. These requirements are intended for writers when adding metadata files (including manifests files and manifest lists) to a table with the given version. +Some tables in this spec have columns that specify requirements for tables by version. These requirements are intended for writers when adding metadata files (including manifests files and manifest lists) to a table with the given version. | Requirement | Write behavior | |-------------|----------------| @@ -135,10 +143,10 @@ Some tables in this spec have columns that specify requirements for v1 and v2 ta | _optional_ | The field can be written or omitted | | _required_ | The field must be written | -Readers should be more permissive because v1 metadata files are allowed in v2 tables so that tables can be upgraded to v2 without rewriting the metadata tree. For manifest list and manifest files, this table shows the expected v2 read behavior: +Readers should be more permissive because v1 metadata files are allowed in v2 tables (or later) so that tables can be upgraded to without rewriting the metadata tree. For manifest list and manifest files, this table shows the expected read behavior for later versions: -| v1 | v2 | v2 read behavior | -|------------|------------|------------------| +| v1 | v2 | v2+ read behavior | +|------------|------------|-------------------| | | _optional_ | Read the field as _optional_ | | | _required_ | Read the field as _optional_; it may be missing in v1 files | | _optional_ | | Ignore the field | @@ -148,12 +156,14 @@ Readers should be more permissive because v1 metadata files are allowed in v2 ta | _required_ | _optional_ | Read the field as _optional_ | | _required_ | _required_ | Fill in a default or throw an exception if the field is missing | -Readers may be more strict for metadata JSON files because the JSON files are not reused and will always match the table version. Required v2 fields that were not present in v1 or optional in v1 may be handled as required fields. For example, a v2 table that is missing `last-sequence-number` can throw an exception. +Readers may be more strict for metadata JSON files because the JSON files are not reused and will always match the table version. Required fields that were not present in or were optional in prior versions may be handled as required fields. For example, a v2 table that is missing `last-sequence-number` can throw an exception. -##### Writing data files +#### Writing data files All columns must be written to data files even if they introduce redundancy with metadata stored in manifest files (e.g. columns with identity partition transforms). Writing all columns provides a backup in case of corruption or bugs in the metadata layer. +Writers are not allowed to commit files with a partition spec that contains a field with an unknown transform. + ### Schemas and Data Types A table's **schema** is a list of named columns. All data types are either primitives or nested types, which are maps, lists, or structs. A table schema is also a struct type. @@ -174,17 +184,18 @@ Supported primitive types are defined in the table below. Primitive types added | Added by version | Primitive type | Description | Requirements | |------------------|--------------------|--------------------------------------------------------------------------|--------------------------------------------------| +| [v3](#version-3) | **`unknown`** | Default / null column type used when a more specific type is not known | Must be optional with `null` defaults; not stored in data files | | | **`boolean`** | True or false | | | | **`int`** | 32-bit signed integers | Can promote to `long` | | | **`long`** | 64-bit signed integers | | | | **`float`** | [32-bit IEEE 754](https://en.wikipedia.org/wiki/IEEE_754) floating point | Can promote to double | | | **`double`** | [64-bit IEEE 754](https://en.wikipedia.org/wiki/IEEE_754) floating point | | -| | **`decimal(P,S)`** | Fixed-point decimal; precision P, scale S | Scale is fixed [1], precision must be 38 or less | +| | **`decimal(P,S)`** | Fixed-point decimal; precision P, scale S | Scale is fixed, precision must be 38 or less | | | **`date`** | Calendar date without timezone or time | | -| | **`time`** | Time of day without date, timezone | Microsecond precision [2] | -| | **`timestamp`** | Timestamp, microsecond precision, without timezone | [2] | +| | **`time`** | Time of day, microsecond precision, without date, timezone | | +| | **`timestamp`** | Timestamp, microsecond precision, without timezone | [1] | | | **`timestamptz`** | Timestamp, microsecond precision, with timezone | [2] | -| [v3](#version-3) | **`timestamp_ns`** | Timestamp, nanosecond precision, without timezone | [2] | +| [v3](#version-3) | **`timestamp_ns`** | Timestamp, nanosecond precision, without timezone | [1] | | [v3](#version-3) | **`timestamptz_ns`** | Timestamp, nanosecond precision, with timezone | [2] | | | **`string`** | Arbitrary-length character sequences | Encoded with UTF-8 [3] | | | **`uuid`** | Universally unique identifiers | Should use 16-byte fixed | @@ -193,10 +204,8 @@ Supported primitive types are defined in the table below. Primitive types added Notes: -1. Decimal scale is fixed and cannot be changed by schema evolution. Precision can only be widened. -2. `time`, `timestamp`, and `timestamptz` values are represented with _microsecond precision_. `timestamp_ns` and `timstamptz_ns` values are represented with _nanosecond precision_. - - Timestamp values _with time zone_ represent a point in time: values are stored as UTC and do not retain a source time zone (`2017-11-16 17:10:34 PST` is stored/retrieved as `2017-11-17 01:10:34 UTC` and these values are considered identical). - - Timestamp values _without time zone_ represent a date and time of day regardless of zone: the time value is independent of zone adjustments (`2017-11-16 17:10:34` is always retrieved as `2017-11-16 17:10:34`). +1. Timestamp values _without time zone_ represent a date and time of day regardless of zone: the time value is independent of zone adjustments (`2017-11-16 17:10:34` is always retrieved as `2017-11-16 17:10:34`). +2. Timestamp values _with time zone_ represent a point in time: values are stored as UTC and do not retain a source time zone (`2017-11-16 17:10:34 PST` is stored/retrieved as `2017-11-17 01:10:34 UTC` and these values are considered identical). 3. Character strings must be stored as UTF-8 encoded byte arrays. For details on how to serialize a schema to JSON, see Appendix C. @@ -213,6 +222,8 @@ The `initial-default` is set only when a field is added to an existing schema. T The `initial-default` and `write-default` produce SQL default value behavior, without rewriting data files. SQL default value behavior when a field is added handles all existing rows as though the rows were written with the new field's default value. Default value changes may only affect future records and all known fields are written into data files. Omitting a known field when writing a data file is never allowed. The write default for a field must be written if a field is not supplied to a write. If the write default for a required field is not set, the writer must fail. +All columns of `unknown` type must default to null. Non-null values for `initial-default` or `write-default` are invalid. + Default values are attributes of fields in schemas and serialized with fields in the JSON format. See [Appendix C](#appendix-c-json-serialization). @@ -222,11 +233,32 @@ Schemas may be evolved by type promotion or adding, deleting, renaming, or reord Evolution applies changes to the table's current schema to produce a new schema that is identified by a unique schema ID, is added to the table's list of schemas, and is set as the table's current schema. -Valid type promotions are: - -* `int` to `long` -* `float` to `double` -* `decimal(P, S)` to `decimal(P', S)` if `P' > P` -- widen the precision of decimal types. +Valid primitive type promotions are: + +| Primitive type | v1, v2 valid type promotions | v3+ valid type promotions | Requirements | +|------------------|------------------------------|------------------------------|--------------| +| `unknown` | | _any type_ | | +| `int` | `long` | `long` | | +| `date` | | `timestamp`, `timestamp_ns` | Promotion to `timestamptz` or `timestamptz_ns` is **not** allowed; values outside the promoted type's range must result in a runtime failure | +| `float` | `double` | `double` | | +| `decimal(P, S)` | `decimal(P', S)` if `P' > P` | `decimal(P', S)` if `P' > P` | Widen precision only | + +Iceberg's Avro manifest format does not store the type of lower and upper bounds, and type promotion does not rewrite existing bounds. For example, when a `float` is promoted to `double`, existing data file bounds are encoded as 4 little-endian bytes rather than 8 little-endian bytes for `double`. To correctly decode the value, the original type at the time the file was written must be inferred according to the following table: + +| Current type | Length of bounds | Inferred type at write time | +|------------------|------------------|-----------------------------| +| `long` | 4 bytes | `int` | +| `long` | 8 bytes | `long` | +| `double` | 4 bytes | `float` | +| `double` | 8 bytes | `double` | +| `timestamp` | 4 bytes | `date` | +| `timestamp` | 8 bytes | `timestamp` | +| `timestamp_ns` | 4 bytes | `date` | +| `timestamp_ns` | 8 bytes | `timestamp_ns` | +| `decimal(P, S)` | _any_ | `decimal(P', S)`; `P' <= P` | + +Type promotion is not allowed for a field that is referenced by `source-id` or `source-ids` of a partition field if the partition transform would produce a different value after promoting the type. For example, `bucket[N]` produces different hash values for `34` and `"34"` (2017239379 != -427558391) but the same value for `34` and `34L`; when an `int` field is the source for a bucket partition field, it may be promoted to `long` but not to `string`. This may happen for the following type promotion cases: +* `date` to `timestamp` or `timestamp_ns` Any struct, including a top-level schema, can evolve through deleting fields, adding new fields, renaming existing fields, reordering existing fields, or promoting a primitive using the valid type promotions. Adding a new field assigns a new ID for that field and for any nested fields. Renaming an existing field must change the name, but not the field ID. Deleting a field removes it from the current schema. Field deletion cannot be rolled back unless the field was nullable or if the current snapshot has not changed. @@ -243,7 +275,7 @@ Struct evolution requires the following rules for default values: * If a field value is missing from a struct's `write-default`, the field's `write-default` must be used for the field -#### Column Projection +##### Column Projection Columns in Iceberg data files are selected by field id. The table schema's column names and order may change after a data file is written, and projection must be done using field ids. @@ -290,16 +322,101 @@ Iceberg tables must not use field ids greater than 2147483447 (`Integer.MAX_VALU The set of metadata columns is: -| Field id, name | Type | Description | -|-----------------------------|---------------|-------------| -| **`2147483646 _file`** | `string` | Path of the file in which a row is stored | -| **`2147483645 _pos`** | `long` | Ordinal position of a row in the source data file | -| **`2147483644 _deleted`** | `boolean` | Whether the row has been deleted | -| **`2147483643 _spec_id`** | `int` | Spec ID used to track the file containing a row | -| **`2147483642 _partition`** | `struct` | Partition to which a row belongs | -| **`2147483546 file_path`** | `string` | Path of a file, used in position-based delete files | -| **`2147483545 pos`** | `long` | Ordinal position of a row, used in position-based delete files | -| **`2147483544 row`** | `struct<...>` | Deleted row values, used in position-based delete files | +| Field id, name | Type | Description | +|----------------------------------|---------------|--------------------------------------------------------------------------------------------------------| +| **`2147483646 _file`** | `string` | Path of the file in which a row is stored | +| **`2147483645 _pos`** | `long` | Ordinal position of a row in the source data file, starting at `0` | +| **`2147483644 _deleted`** | `boolean` | Whether the row has been deleted | +| **`2147483643 _spec_id`** | `int` | Spec ID used to track the file containing a row | +| **`2147483642 _partition`** | `struct` | Partition to which a row belongs | +| **`2147483546 file_path`** | `string` | Path of a file, used in position-based delete files | +| **`2147483545 pos`** | `long` | Ordinal position of a row, used in position-based delete files | +| **`2147483544 row`** | `struct<...>` | Deleted row values, used in position-based delete files | +| **`2147483543 _row_id`** | `long` | A unique long assigned when row-lineage is enabled, see [Row Lineage](#row-lineage) | +| **`2147483542 _last_updated_sequence_number`** | `long` | The sequence number which last updated this row when row-lineage is enabled [Row Lineage](#row-lineage) | + +#### Row Lineage + +In v3 and later, an Iceberg table can track row lineage fields for all newly created rows. Row lineage is enabled by setting the field `row-lineage` to true in the table's metadata. When enabled, engines must maintain the `next-row-id` table field and the following row-level fields when writing data files: + +* `_row_id` a unique long identifier for every row within the table. The value is assigned via inheritance when a row is first added to the table and the existing value is explicitly written when the row is copied into a new file. +* `_last_updated_sequence_number` the sequence number of the commit that last updated a row. The value is inherited when a row is first added or modified and the existing value is explicitly written when the row is written to a different data file but not modified. + +These fields are assigned and updated by inheritance because the commit sequence number and starting row ID are not assigned until the snapshot is successfully committed. Inheritance is used to allow writing data and manifest files before values are known so that it is not necessary to rewrite data and manifest files when an optimistic commit is retried. + +When row lineage is enabled, new snapshots cannot include [Equality Deletes](#equality-delete-files). Row lineage is incompatible with equality deletes because lineage values must be maintained, but equality deletes are used to avoid reading existing data before writing changes. + + +##### Row lineage assignment + +Row lineage fields are written when row lineage is enabled. When not enabled, row lineage fields (`_row_id` and `_last_updated_sequence_number`) must not be written to data files. The rest of this section applies when row lineage is enabled. + +When a row is added or modified, the `_last_updated_sequence_number` field is set to `null` so that it is inherited when reading. Similarly, the `_row_id` field for an added row is set to `null` and assigned when reading. + +A data file with only new rows for the table may omit the `_last_updated_sequence_number` and `_row_id`. If the columns are missing, readers should treat both columns as if they exist and are set to null for all rows. + +On read, if `_last_updated_sequence_number` is `null` it is assigned the `sequence_number` of the data file's manifest entry. The data sequence number of a data file is documented in [Sequence Number Inheritance](#sequence-number-inheritance). + +When `null`, a row's `_row_id` field is assigned to the `first_row_id` from its containing data file plus the row position in that data file (`_pos`). A data file's `first_row_id` field is assigned using inheritance and is documented in [First Row ID Inheritance](#first-row-id-inheritance). A manifest's `first_row_id` is assigned when writing the manifest list for a snapshot and is documented in [First Row ID Assignment](#first-row-id-assignment). A snapshot's `first-row-id` is set to the table's `next-row-id` and is documented in [Snapshot Row IDs](#snapshot-row-ids). + +Values for `_row_id` and `_last_updated_sequence_number` are either read from the data file or assigned at read time. As a result on read, rows in a table always have non-null values for these fields when lineage is enabled. + +When an existing row is moved to a different data file for any reason, writers are required to write `_row_id` and `_last_updated_sequence_number` according to the following rules: + +1. The row's existing non-null `_row_id` must be copied into the new data file +2. If the write has modified the row, the `_last_updated_sequence_number` field must be set to `null` (so that the modification's sequence number replaces the current value) +3. If the write has not modified the row, the existing non-null `_last_updated_sequence_number` value must be copied to the new data file + + +##### Row lineage example + +This example demonstrates how `_row_id` and `_last_updated_sequence_number` are assigned for a snapshot when row lineage is enabled. This starts with a table with row lineage enabled and a `next-row-id` of 1000. + +Writing a new append snapshot would create snapshot metadata with `first-row-id` assigned to the table's `next-row-id`: + +```json +{ + "operation": "append", + "first-row-id": 1000, + ... +} +``` + +The snapshot's manifest list would contain existing manifests, plus new manifests with an assigned `first_row_id` based on the `added_rows_count` of previously listed added manifests: + +| `manifest_path` | `added_rows_count` | `existing_rows_count` | `first_row_id` | +|-----------------|--------------------|-----------------------|--------------------| +| ... | ... | ... | ... | +| existing | 75 | 0 | 925 | +| added1 | 100 | 25 | 1000 | +| added2 | 0 | 100 | 1100 | +| added3 | 125 | 25 | 1100 | + +The first added file, `added1`, is assigned the same `first_row_id` as the snapshot and the following manifests are assigned `first_row_id` based on the number of rows added by the previously listed manifests. The second file, `added2`, does not change the `first_row_id` of the next manifest because it contains no added data files. + +Within `added1`, the first added manifest, each data file's `first_row_id` follows a similar pattern: + +| `status` | `file_path` | `record_count` | `first_row_id` | +|----------|-------------|----------------|----------------| +| EXISTING | data1 | 25 | 800 | +| ADDED | data2 | 50 | null (1000) | +| ADDED | data3 | 50 | null (1050) | + +The `first_row_id` of the EXISTING file `data1` was already assigned, so the file metadata was copied into manifest `added1`. + +Files `data2` and `data3` are written with `null` for `first_row_id` and are assigned `first_row_id` at read time based on the manifest's `first_row_id` and the `record_count` of previously listed ADDED files in this manifest: (1,000 + 0) and (1,000 + 50). + +When the new snapshot is committed, the table's `next-row-id` must also be updated (even if the new snapshot is not in the main branch). Because 225 rows were added (`added1`: 100 + `added2`: 0 + `added3`: 125), the new value is 1,000 + 225 = 1,225: + + +##### Enabling Row Lineage for Non-empty Tables + +Any snapshot without the field `first-row-id` does not have any lineage information and values for `_row_id` and `_last_updated_sequence_number` cannot be assigned accurately. + +All files that were added before `row-lineage` was enabled should propagate null for all of the `row-lineage` related +fields. The values for `_row_id` and `_last_updated_sequence_number` should always return null and when these rows are copied, +null should be explicitly written. After this point, rows are treated as if they were just created +and assigned `row_id` and `_last_updated_sequence_number` as if they were new rows. ### Partitioning @@ -317,7 +434,9 @@ The source columns, selected by ids, must be a primitive type and cannot be cont Partition specs capture the transform from table data to partition values. This is used to transform predicates to partition predicates, in addition to transforming data values. Deriving partition predicates from column predicates on the table data is used to separate the logical queries from physical storage: the partitioning can change and the correct partition filters are always derived from column predicates. This simplifies queries because users don’t have to supply both logical predicates and partition predicates. For more information, see Scan Planning below. -Two partition specs are considered equivalent with each other if they have the same number of fields and for each corresponding field, the fields have the same source column ID, transform definition and partition name. Writers must not create a new parition spec if there already exists a compatible partition spec defined in the table. +Partition fields that use an unknown transform can be read by ignoring the partition field for the purpose of filtering data files during scan planning. In v1 and v2, readers should ignore fields with unknown transforms while reading; this behavior is required in v3. Writers are not allowed to commit data using a partition spec that contains a field with an unknown transform. + +Two partition specs are considered equivalent with each other if they have the same number of fields and for each corresponding field, the fields have the same source column IDs, transform definition and partition name. Writers must not create a new parition spec if there already exists a compatible partition spec defined in the table. Partition field IDs must be reused if an existing partition spec contains an equivalent field. @@ -444,29 +563,29 @@ The schema of a manifest file is a struct called `manifest_entry` with the follo `data_file` is a struct with the following fields: -| v1 | v2 | Field id, name | Type | Description | -| ---------- | ---------- |-----------------------------------|------------------------------|-------------| -| | _required_ | **`134 content`** | `int` with meaning: `0: DATA`, `1: POSITION DELETES`, `2: EQUALITY DELETES` | Type of content stored by the data file: data, equality deletes, or position deletes (all v1 files are data files) | -| _required_ | _required_ | **`100 file_path`** | `string` | Full URI for the file with FS scheme | -| _required_ | _required_ | **`101 file_format`** | `string` | String file format name, avro, orc or parquet | -| _required_ | _required_ | **`102 partition`** | `struct<...>` | Partition data tuple, schema based on the partition spec output using partition field ids for the struct field ids | -| _required_ | _required_ | **`103 record_count`** | `long` | Number of records in this file | -| _required_ | _required_ | **`104 file_size_in_bytes`** | `long` | Total file size in bytes | -| _required_ | | ~~**`105 block_size_in_bytes`**~~ | `long` | **Deprecated. Always write a default in v1. Do not write in v2.** | -| _optional_ | | ~~**`106 file_ordinal`**~~ | `int` | **Deprecated. Do not write.** | -| _optional_ | | ~~**`107 sort_columns`**~~ | `list<112: int>` | **Deprecated. Do not write.** | -| _optional_ | _optional_ | **`108 column_sizes`** | `map<117: int, 118: long>` | Map from column id to the total size on disk of all regions that store the column. Does not include bytes necessary to read other columns, like footers. Leave null for row-oriented formats (Avro) | -| _optional_ | _optional_ | **`109 value_counts`** | `map<119: int, 120: long>` | Map from column id to number of values in the column (including null and NaN values) | -| _optional_ | _optional_ | **`110 null_value_counts`** | `map<121: int, 122: long>` | Map from column id to number of null values in the column | -| _optional_ | _optional_ | **`137 nan_value_counts`** | `map<138: int, 139: long>` | Map from column id to number of NaN values in the column | -| _optional_ | _optional_ | **`111 distinct_counts`** | `map<123: int, 124: long>` | Map from column id to number of distinct values in the column; distinct counts must be derived using values in the file by counting or using sketches, but not using methods like merging existing distinct counts | -| _optional_ | _optional_ | **`125 lower_bounds`** | `map<126: int, 127: binary>` | Map from column id to lower bound in the column serialized as binary [1]. Each value must be less than or equal to all non-null, non-NaN values in the column for the file [2] | -| _optional_ | _optional_ | **`128 upper_bounds`** | `map<129: int, 130: binary>` | Map from column id to upper bound in the column serialized as binary [1]. Each value must be greater than or equal to all non-null, non-Nan values in the column for the file [2] | -| _optional_ | _optional_ | **`131 key_metadata`** | `binary` | Implementation-specific key metadata for encryption | -| _optional_ | _optional_ | **`132 split_offsets`** | `list<133: long>` | Split offsets for the data file. For example, all row group offsets in a Parquet file. Must be sorted ascending | -| | _optional_ | **`135 equality_ids`** | `list<136: int>` | Field ids used to determine row equality in equality delete files. Required when `content=2` and should be null otherwise. Fields with ids listed in this column must be present in the delete file | -| _optional_ | _optional_ | **`140 sort_order_id`** | `int` | ID representing sort order for this file [3]. | - +| v1 | v2 | v3 | Field id, name | Type | Description | +| ---------- |------------|------------|-----------------------------------|-----------------------------------------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| | _required_ | _required_ | **`134 content`** | `int` with meaning: `0: DATA`, `1: POSITION DELETES`, `2: EQUALITY DELETES` | Type of content stored by the data file: data, equality deletes, or position deletes (all v1 files are data files) | +| _required_ | _required_ | _required_ | **`100 file_path`** | `string` | Full URI for the file with FS scheme | +| _required_ | _required_ | _required_ | **`101 file_format`** | `string` | String file format name, avro, orc or parquet | +| _required_ | _required_ | _required_ | **`102 partition`** | `struct<...>` | Partition data tuple, schema based on the partition spec output using partition field ids for the struct field ids | +| _required_ | _required_ | _required_ | **`103 record_count`** | `long` | Number of records in this file | +| _required_ | _required_ | _required_ | **`104 file_size_in_bytes`** | `long` | Total file size in bytes | +| _required_ | | | ~~**`105 block_size_in_bytes`**~~ | `long` | **Deprecated. Always write a default in v1. Do not write in v2 or v3.** | +| _optional_ | | | ~~**`106 file_ordinal`**~~ | `int` | **Deprecated. Do not write.** | +| _optional_ | | | ~~**`107 sort_columns`**~~ | `list<112: int>` | **Deprecated. Do not write.** | +| _optional_ | _optional_ | _optional_ | **`108 column_sizes`** | `map<117: int, 118: long>` | Map from column id to the total size on disk of all regions that store the column. Does not include bytes necessary to read other columns, like footers. Leave null for row-oriented formats (Avro) | +| _optional_ | _optional_ | _optional_ | **`109 value_counts`** | `map<119: int, 120: long>` | Map from column id to number of values in the column (including null and NaN values) | +| _optional_ | _optional_ | _optional_ | **`110 null_value_counts`** | `map<121: int, 122: long>` | Map from column id to number of null values in the column | +| _optional_ | _optional_ | _optional_ | **`137 nan_value_counts`** | `map<138: int, 139: long>` | Map from column id to number of NaN values in the column | +| _optional_ | _optional_ | _optional_ | **`111 distinct_counts`** | `map<123: int, 124: long>` | Map from column id to number of distinct values in the column; distinct counts must be derived using values in the file by counting or using sketches, but not using methods like merging existing distinct counts | +| _optional_ | _optional_ | _optional_ | **`125 lower_bounds`** | `map<126: int, 127: binary>` | Map from column id to lower bound in the column serialized as binary [1]. Each value must be less than or equal to all non-null, non-NaN values in the column for the file [2] | +| _optional_ | _optional_ | _optional_ | **`128 upper_bounds`** | `map<129: int, 130: binary>` | Map from column id to upper bound in the column serialized as binary [1]. Each value must be greater than or equal to all non-null, non-Nan values in the column for the file [2] | +| _optional_ | _optional_ | _optional_ | **`131 key_metadata`** | `binary` | Implementation-specific key metadata for encryption | +| _optional_ | _optional_ | _optional_ | **`132 split_offsets`** | `list<133: long>` | Split offsets for the data file. For example, all row group offsets in a Parquet file. Must be sorted ascending | +| | _optional_ | _optional_ | **`135 equality_ids`** | `list<136: int>` | Field ids used to determine row equality in equality delete files. Required when `content=2` and should be null otherwise. Fields with ids listed in this column must be present in the delete file | +| _optional_ | _optional_ | _optional_ | **`140 sort_order_id`** | `int` | ID representing sort order for this file [3]. | +| | | _optional_ | **`142 first_row_id`** | `long` | The `_row_id` for the first row in the data file. See [First Row ID Inheritance](#first-row-id-inheritance) | Notes: 1. Single-value serialization for lower and upper bounds is detailed in Appendix D. @@ -510,21 +629,31 @@ Inheriting sequence numbers through the metadata tree allows writing a new manif When reading v1 manifests with no sequence number column, sequence numbers for all files must default to 0. +#### First Row ID Inheritance + +Row ID inheritance is used when row lineage is enabled. When not enabled, a data file's `first_row_id` must always be set to `null`. The rest of this section applies when row lineage is enabled. + +When adding a new data file, its `first_row_id` field is set to `null` because it is not assigned until the snapshot is successfully committed. + +When reading, the `first_row_id` is assigned by replacing `null` with the manifest's `first_row_id` plus the sum of `record_count` for all added data files that preceded the file in the manifest. + +The `first_row_id` is only inherited for added data files. The inherited value must be written into the data file metadata for existing and deleted entries. The value of `first_row_id` for delete files is always `null`. ### Snapshots A snapshot consists of the following fields: -| v1 | v2 | Field | Description | -| ---------- | ---------- | ------------------------ | ----------- | -| _required_ | _required_ | **`snapshot-id`** | A unique long ID | -| _optional_ | _optional_ | **`parent-snapshot-id`** | The snapshot ID of the snapshot's parent. Omitted for any snapshot with no parent | -| | _required_ | **`sequence-number`** | A monotonically increasing long that tracks the order of changes to a table | -| _required_ | _required_ | **`timestamp-ms`** | A timestamp when the snapshot was created, used for garbage collection and table inspection | -| _optional_ | _required_ | **`manifest-list`** | The location of a manifest list for this snapshot that tracks manifest files with additional metadata | -| _optional_ | | **`manifests`** | A list of manifest file locations. Must be omitted if `manifest-list` is present | -| _optional_ | _required_ | **`summary`** | A string map that summarizes the snapshot changes, including `operation` (see below) | -| _optional_ | _optional_ | **`schema-id`** | ID of the table's current schema when the snapshot was created | +| v1 | v2 | v3 | Field | Description | +| ---------- | ---------- |------------|------------------------------|------------------------------------------------------------------------------------------------------------------------------------| +| _required_ | _required_ | _required_ | **`snapshot-id`** | A unique long ID | +| _optional_ | _optional_ | _optional_ | **`parent-snapshot-id`** | The snapshot ID of the snapshot's parent. Omitted for any snapshot with no parent | +| | _required_ | _required_ | **`sequence-number`** | A monotonically increasing long that tracks the order of changes to a table | +| _required_ | _required_ | _required_ | **`timestamp-ms`** | A timestamp when the snapshot was created, used for garbage collection and table inspection | +| _optional_ | _required_ | _required_ | **`manifest-list`** | The location of a manifest list for this snapshot that tracks manifest files with additional metadata | +| _optional_ | | | **`manifests`** | A list of manifest file locations. Must be omitted if `manifest-list` is present | +| _optional_ | _required_ | _required_ | **`summary`** | A string map that summarizes the snapshot changes, including `operation` (see below) | +| _optional_ | _optional_ | _optional_ | **`schema-id`** | ID of the table's current schema when the snapshot was created | +| | | _optional_ | **`first-row-id`** | The first `_row_id` assigned to the first row in the first data file in the first manifest, see [Row Lineage](#row-lineage) | The snapshot summary's `operation` field is used by some operations, like snapshot expiration, to skip processing certain snapshots. Possible `operation` values are: @@ -544,7 +673,16 @@ Manifests for a snapshot are tracked by a manifest list. Valid snapshots are stored as a list in table metadata. For serialization, see Appendix C. -#### Manifest Lists +#### Snapshot Row IDs + +When row lineage is not enabled, `first-row-id` must be omitted. The rest of this section applies when row lineage is enabled. + +A snapshot's `first-row-id` is assigned to the table's current `next-row-id` on each commit attempt. If a commit is retried, the `first-row-id` must be reassigned. If a commit contains no new rows, `first-row-id` should be omitted. + +The snapshot's `first-row-id` is the starting `first_row_id` assigned to manifests in the snapshot's manifest list. + + +### Manifest Lists Snapshots are embedded in table metadata, but the list of manifests for a snapshot are stored in a separate manifest list file. @@ -556,23 +694,24 @@ A manifest list is a valid Iceberg data file: files must use valid Iceberg forma Manifest list files store `manifest_file`, a struct with the following fields: -| v1 | v2 | Field id, name | Type | Description | -| ---------- | ---------- |--------------------------------|---------------------------------------------|-------------| -| _required_ | _required_ | **`500 manifest_path`** | `string` | Location of the manifest file | -| _required_ | _required_ | **`501 manifest_length`** | `long` | Length of the manifest file in bytes | -| _required_ | _required_ | **`502 partition_spec_id`** | `int` | ID of a partition spec used to write the manifest; must be listed in table metadata `partition-specs` | -| | _required_ | **`517 content`** | `int` with meaning: `0: data`, `1: deletes` | The type of files tracked by the manifest, either data or delete files; 0 for all v1 manifests | -| | _required_ | **`515 sequence_number`** | `long` | The sequence number when the manifest was added to the table; use 0 when reading v1 manifest lists | -| | _required_ | **`516 min_sequence_number`** | `long` | The minimum data sequence number of all live data or delete files in the manifest; use 0 when reading v1 manifest lists | -| _required_ | _required_ | **`503 added_snapshot_id`** | `long` | ID of the snapshot where the manifest file was added | -| _optional_ | _required_ | **`504 added_files_count`** | `int` | Number of entries in the manifest that have status `ADDED` (1), when `null` this is assumed to be non-zero | -| _optional_ | _required_ | **`505 existing_files_count`** | `int` | Number of entries in the manifest that have status `EXISTING` (0), when `null` this is assumed to be non-zero | -| _optional_ | _required_ | **`506 deleted_files_count`** | `int` | Number of entries in the manifest that have status `DELETED` (2), when `null` this is assumed to be non-zero | -| _optional_ | _required_ | **`512 added_rows_count`** | `long` | Number of rows in all of files in the manifest that have status `ADDED`, when `null` this is assumed to be non-zero | -| _optional_ | _required_ | **`513 existing_rows_count`** | `long` | Number of rows in all of files in the manifest that have status `EXISTING`, when `null` this is assumed to be non-zero | -| _optional_ | _required_ | **`514 deleted_rows_count`** | `long` | Number of rows in all of files in the manifest that have status `DELETED`, when `null` this is assumed to be non-zero | -| _optional_ | _optional_ | **`507 partitions`** | `list<508: field_summary>` (see below) | A list of field summaries for each partition field in the spec. Each field in the list corresponds to a field in the manifest file’s partition spec. | -| _optional_ | _optional_ | **`519 key_metadata`** | `binary` | Implementation-specific key metadata for encryption | +| v1 | v2 | v3 | Field id, name | Type | Description | +| ---------- | ---------- |------------|----------------------------------|---------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------| +| _required_ | _required_ | _required_ | **`500 manifest_path`** | `string` | Location of the manifest file | +| _required_ | _required_ | _required_ | **`501 manifest_length`** | `long` | Length of the manifest file in bytes | +| _required_ | _required_ | _required_ | **`502 partition_spec_id`** | `int` | ID of a partition spec used to write the manifest; must be listed in table metadata `partition-specs` | +| | _required_ | _required_ | **`517 content`** | `int` with meaning: `0: data`, `1: deletes` | The type of files tracked by the manifest, either data or delete files; 0 for all v1 manifests | +| | _required_ | _required_ | **`515 sequence_number`** | `long` | The sequence number when the manifest was added to the table; use 0 when reading v1 manifest lists | +| | _required_ | _required_ | **`516 min_sequence_number`** | `long` | The minimum data sequence number of all live data or delete files in the manifest; use 0 when reading v1 manifest lists | +| _required_ | _required_ | _required_ | **`503 added_snapshot_id`** | `long` | ID of the snapshot where the manifest file was added | +| _optional_ | _required_ | _required_ | **`504 added_files_count`** | `int` | Number of entries in the manifest that have status `ADDED` (1), when `null` this is assumed to be non-zero | +| _optional_ | _required_ | _required_ | **`505 existing_files_count`** | `int` | Number of entries in the manifest that have status `EXISTING` (0), when `null` this is assumed to be non-zero | +| _optional_ | _required_ | _required_ | **`506 deleted_files_count`** | `int` | Number of entries in the manifest that have status `DELETED` (2), when `null` this is assumed to be non-zero | +| _optional_ | _required_ | _required_ | **`512 added_rows_count`** | `long` | Number of rows in all of files in the manifest that have status `ADDED`, when `null` this is assumed to be non-zero | +| _optional_ | _required_ | _required_ | **`513 existing_rows_count`** | `long` | Number of rows in all of files in the manifest that have status `EXISTING`, when `null` this is assumed to be non-zero | +| _optional_ | _required_ | _required_ | **`514 deleted_rows_count`** | `long` | Number of rows in all of files in the manifest that have status `DELETED`, when `null` this is assumed to be non-zero | +| _optional_ | _optional_ | _optional_ | **`507 partitions`** | `list<508: field_summary>` (see below) | A list of field summaries for each partition field in the spec. Each field in the list corresponds to a field in the manifest file’s partition spec. | +| _optional_ | _optional_ | _optional_ | **`519 key_metadata`** | `binary` | Implementation-specific key metadata for encryption | +| | | _optional_ | **`520 first_row_id`** | `long` | The starting `_row_id` to assign to rows added by `ADDED` data files [First Row ID Assignment](#first-row-id-assignment) | `field_summary` is a struct with the following fields: @@ -588,7 +727,15 @@ Notes: 1. Lower and upper bounds are serialized to bytes using the single-object serialization in Appendix D. The type of used to encode the value is the type of the partition field data. 2. If -0.0 is a value of the partition field, the `lower_bound` must not be +0.0, and if +0.0 is a value of the partition field, the `upper_bound` must not be -0.0. -#### Scan Planning +#### First Row ID Assignment + +Row ID inheritance is used when row lineage is enabled. When not enabled, a manifest's `first_row_id` must always be set to `null`. Once enabled, row lineage cannot be disabled. The rest of this section applies when row lineage is enabled. + +When adding a new data manifest file, its `first_row_id` field is assigned the value of the snapshot's `first_row_id` plus the sum of `added_rows_count` for all data manifests that preceded the manifest in the manifest list. + +The `first_row_id` is only assigned for new data manifests. Values for existing manifests must be preserved when writing a new manifest list. The value of `first_row_id` for delete manifests is always `null`. + +### Scan Planning Scans are planned by reading the manifest files for the current snapshot. Deleted entries in data and delete manifests (those marked with status "DELETED") are not used in a scan. @@ -600,6 +747,8 @@ Scan predicates are converted to partition predicates using an _inclusive projec For example, an `events` table with a timestamp column named `ts` that is partitioned by `ts_day=day(ts)` is queried by users with ranges over the timestamp column: `ts > X`. The inclusive projection is `ts_day >= day(X)`, which is used to select files that may have matching rows. Note that, in most cases, timestamps just before `X` will be included in the scan because the file contains rows that match the predicate and rows that do not match the predicate. +The inclusive projection for an unknown partition transform is _true_ because the partition field is ignored and not used in filtering. + Scan predicates are also used to filter data and delete files using column bounds and counts that are stored by field id in manifests. The same filter logic can be used for both data and delete files because both store metrics of the rows either inserted or deleted. If metrics show that a delete file has no rows that match a scan predicate, it may be ignored just as a data file would be ignored [2]. Data files that match the query filter must be read by the scan. @@ -611,10 +760,10 @@ Delete files that match the query filter must be applied to data files at read t * A _position_ delete file must be applied to a data file when all of the following are true: - The data file's data sequence number is _less than or equal to_ the delete file's data sequence number - - The data file's partition (both spec and partition values) is equal to the delete file's partition + - The data file's partition (both spec and partition values) is equal [4] to the delete file's partition * An _equality_ delete file must be applied to a data file when all of the following are true: - The data file's data sequence number is _strictly less than_ the delete's data sequence number - - The data file's partition (both spec id and partition values) is equal to the delete file's partition _or_ the delete file's partition spec is unpartitioned + - The data file's partition (both spec id and partition values) is equal [4] to the delete file's partition _or_ the delete file's partition spec is unpartitioned In general, deletes are applied only to data files that are older and in the same partition, except for two special cases: @@ -627,8 +776,9 @@ Notes: 1. An alternative, *strict projection*, creates a partition predicate that will match a file if all of the rows in the file must match the scan predicate. These projections are used to calculate the residual predicates for each file in a scan. 2. For example, if `file_a` has rows with `id` between 1 and 10 and a delete file contains rows with `id` between 1 and 4, a scan for `id = 9` may ignore the delete file because none of the deletes can match a row that will be selected. 3. Floating point partition values are considered equal if their IEEE 754 floating-point "single format" bit layout are equal with NaNs normalized to have only the the most significant mantissa bit set (the equivelant of calling `Float.floatToIntBits` or `Double.doubleToLongBits` in Java). The Avro specification requires all floating point values to be encoded in this format. +4. Unknown partition transforms do not affect partition equality. Although partition fields with unknown transforms are ignored for filtering, the result of an unknown transform is still used when testing whether partition values are equal. -#### Snapshot Reference +### Snapshot References Iceberg tables keep track of branches and tags using snapshot references. Tags are labels for individual snapshots. Branches are mutable named references that can be updated by committing a new snapshot as the branch's referenced snapshot using the [Commit Conflict Resolution and Retry](#commit-conflict-resolution-and-retry) procedures. @@ -645,7 +795,7 @@ The snapshot reference object records all the information of a reference includi Valid snapshot references are stored as the values of the `refs` map in table metadata. For serialization, see Appendix C. -#### Snapshot Retention Policy +### Snapshot Retention Policy Table snapshots expire and are removed from metadata to allow removed or replaced data files to be physically deleted. The snapshot expiration procedure removes snapshots from table metadata and applies the table's retention policy. @@ -671,34 +821,38 @@ The atomic operation used to commit metadata depends on how tables are tracked a Table metadata consists of the following fields: -| v1 | v2 | Field | Description | -| ---------- | ---------- | ----- | ----------- | -| _required_ | _required_ | **`format-version`** | An integer version number for the format. Currently, this can be 1 or 2 based on the spec. Implementations must throw an exception if a table's version is higher than the supported version. | -| _optional_ | _required_ | **`table-uuid`** | A UUID that identifies the table, generated when the table is created. Implementations must throw an exception if a table's UUID does not match the expected UUID after refreshing metadata. | -| _required_ | _required_ | **`location`**| The table's base location. This is used by writers to determine where to store data files, manifest files, and table metadata files. | -| | _required_ | **`last-sequence-number`**| The table's highest assigned sequence number, a monotonically increasing long that tracks the order of snapshots in a table. | -| _required_ | _required_ | **`last-updated-ms`**| Timestamp in milliseconds from the unix epoch when the table was last updated. Each table metadata file should update this field just before writing. | -| _required_ | _required_ | **`last-column-id`**| An integer; the highest assigned column ID for the table. This is used to ensure columns are always assigned an unused ID when evolving schemas. | -| _required_ | | **`schema`**| The table’s current schema. (**Deprecated**: use `schemas` and `current-schema-id` instead) | -| _optional_ | _required_ | **`schemas`**| A list of schemas, stored as objects with `schema-id`. | -| _optional_ | _required_ | **`current-schema-id`**| ID of the table's current schema. | -| _required_ | | **`partition-spec`**| The table’s current partition spec, stored as only fields. Note that this is used by writers to partition data, but is not used when reading because reads use the specs stored in manifest files. (**Deprecated**: use `partition-specs` and `default-spec-id` instead) | -| _optional_ | _required_ | **`partition-specs`**| A list of partition specs, stored as full partition spec objects. | -| _optional_ | _required_ | **`default-spec-id`**| ID of the "current" spec that writers should use by default. | -| _optional_ | _required_ | **`last-partition-id`**| An integer; the highest assigned partition field ID across all partition specs for the table. This is used to ensure partition fields are always assigned an unused ID when evolving specs. | -| _optional_ | _optional_ | **`properties`**| A string to string map of table properties. This is used to control settings that affect reading and writing and is not intended to be used for arbitrary metadata. For example, `commit.retry.num-retries` is used to control the number of commit retries. | -| _optional_ | _optional_ | **`current-snapshot-id`**| `long` ID of the current table snapshot; must be the same as the current ID of the `main` branch in `refs`. | -| _optional_ | _optional_ | **`snapshots`**| A list of valid snapshots. Valid snapshots are snapshots for which all data files exist in the file system. A data file must not be deleted from the file system until the last snapshot in which it was listed is garbage collected. | -| _optional_ | _optional_ | **`snapshot-log`**| A list (optional) of timestamp and snapshot ID pairs that encodes changes to the current snapshot for the table. Each time the current-snapshot-id is changed, a new entry should be added with the last-updated-ms and the new current-snapshot-id. When snapshots are expired from the list of valid snapshots, all entries before a snapshot that has expired should be removed. | -| _optional_ | _optional_ | **`metadata-log`**| A list (optional) of timestamp and metadata file location pairs that encodes changes to the previous metadata files for the table. Each time a new metadata file is created, a new entry of the previous metadata file location should be added to the list. Tables can be configured to remove oldest metadata log entries and keep a fixed-size log of the most recent entries after a commit. | -| _optional_ | _required_ | **`sort-orders`**| A list of sort orders, stored as full sort order objects. | -| _optional_ | _required_ | **`default-sort-order-id`**| Default sort order id of the table. Note that this could be used by writers, but is not used when reading because reads use the specs stored in manifest files. | -| | _optional_ | **`refs`** | A map of snapshot references. The map keys are the unique snapshot reference names in the table, and the map values are snapshot reference objects. There is always a `main` branch reference pointing to the `current-snapshot-id` even if the `refs` map is null. | -| _optional_ | _optional_ | **`statistics`** | A list (optional) of [table statistics](#table-statistics). | -| _optional_ | _optional_ | **`partition-statistics`** | A list (optional) of [partition statistics](#partition-statistics). | +| v1 | v2 | v3 | Field | Description | +| ---------- | ---------- |------------|-----------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| _required_ | _required_ | _required_ | **`format-version`** | An integer version number for the format. Currently, this can be 1 or 2 based on the spec. Implementations must throw an exception if a table's version is higher than the supported version. | +| _optional_ | _required_ | _required_ | **`table-uuid`** | A UUID that identifies the table, generated when the table is created. Implementations must throw an exception if a table's UUID does not match the expected UUID after refreshing metadata. | +| _required_ | _required_ | _required_ | **`location`** | The table's base location. This is used by writers to determine where to store data files, manifest files, and table metadata files. | +| | _required_ | _required_ | **`last-sequence-number`** | The table's highest assigned sequence number, a monotonically increasing long that tracks the order of snapshots in a table. | +| _required_ | _required_ | _required_ | **`last-updated-ms`** | Timestamp in milliseconds from the unix epoch when the table was last updated. Each table metadata file should update this field just before writing. | +| _required_ | _required_ | _required_ | **`last-column-id`** | An integer; the highest assigned column ID for the table. This is used to ensure columns are always assigned an unused ID when evolving schemas. | +| _required_ | | | **`schema`** | The table’s current schema. (**Deprecated**: use `schemas` and `current-schema-id` instead) | +| _optional_ | _required_ | _required_ | **`schemas`** | A list of schemas, stored as objects with `schema-id`. | +| _optional_ | _required_ | _required_ | **`current-schema-id`** | ID of the table's current schema. | +| _required_ | | | **`partition-spec`** | The table’s current partition spec, stored as only fields. Note that this is used by writers to partition data, but is not used when reading because reads use the specs stored in manifest files. (**Deprecated**: use `partition-specs` and `default-spec-id` instead) | +| _optional_ | _required_ | _required_ | **`partition-specs`** | A list of partition specs, stored as full partition spec objects. | +| _optional_ | _required_ | _required_ | **`default-spec-id`** | ID of the "current" spec that writers should use by default. | +| _optional_ | _required_ | _required_ | **`last-partition-id`** | An integer; the highest assigned partition field ID across all partition specs for the table. This is used to ensure partition fields are always assigned an unused ID when evolving specs. | +| _optional_ | _optional_ | _optional_ | **`properties`** | A string to string map of table properties. This is used to control settings that affect reading and writing and is not intended to be used for arbitrary metadata. For example, `commit.retry.num-retries` is used to control the number of commit retries. | +| _optional_ | _optional_ | _optional_ | **`current-snapshot-id`** | `long` ID of the current table snapshot; must be the same as the current ID of the `main` branch in `refs`. | +| _optional_ | _optional_ | _optional_ | **`snapshots`** | A list of valid snapshots. Valid snapshots are snapshots for which all data files exist in the file system. A data file must not be deleted from the file system until the last snapshot in which it was listed is garbage collected. | +| _optional_ | _optional_ | _optional_ | **`snapshot-log`** | A list (optional) of timestamp and snapshot ID pairs that encodes changes to the current snapshot for the table. Each time the current-snapshot-id is changed, a new entry should be added with the last-updated-ms and the new current-snapshot-id. When snapshots are expired from the list of valid snapshots, all entries before a snapshot that has expired should be removed. | +| _optional_ | _optional_ | _optional_ | **`metadata-log`** | A list (optional) of timestamp and metadata file location pairs that encodes changes to the previous metadata files for the table. Each time a new metadata file is created, a new entry of the previous metadata file location should be added to the list. Tables can be configured to remove oldest metadata log entries and keep a fixed-size log of the most recent entries after a commit. | +| _optional_ | _required_ | _required_ | **`sort-orders`** | A list of sort orders, stored as full sort order objects. | +| _optional_ | _required_ | _required_ | **`default-sort-order-id`** | Default sort order id of the table. Note that this could be used by writers, but is not used when reading because reads use the specs stored in manifest files. | +| | _optional_ | _optional_ | **`refs`** | A map of snapshot references. The map keys are the unique snapshot reference names in the table, and the map values are snapshot reference objects. There is always a `main` branch reference pointing to the `current-snapshot-id` even if the `refs` map is null. | +| _optional_ | _optional_ | _optional_ | **`statistics`** | A list (optional) of [table statistics](#table-statistics). | +| _optional_ | _optional_ | _optional_ | **`partition-statistics`** | A list (optional) of [partition statistics](#partition-statistics). | +| | | _optional_ | **`row-lineage`** | A boolean, defaulting to false, setting whether or not to track the creation and updates to rows in the table. See [Row Lineage](#row-lineage). | +| | | _optional_ | **`next-row-id`** | A value higher than all assigned row IDs; the next snapshot's `first-row-id`. See [Row Lineage](#row-lineage). | For serialization details, see Appendix C. +When a new snapshot is added, the table's `next-row-id` should be updated to the previous `next-row-id` plus the sum of `record_count` for all data files added in the snapshot (this is also equal to the sum of `added_rows_count` for all manifests added in the snapshot). This ensures that `next-row-id` is always higher than any assigned row ID in the table. + #### Table Statistics Table statistics files are valid [Puffin files](puffin-spec.md). Statistics are informational. A reader can choose to @@ -743,7 +897,7 @@ Partition statistics file must be registered in the table metadata file to be co | _required_ | _required_ | **`statistics-path`** | `string` | Path of the partition statistics file. See [Partition statistics file](#partition-statistics-file). | | _required_ | _required_ | **`file-size-in-bytes`** | `long` | Size of the partition statistics file. | -#### Partition Statistics File +##### Partition Statistics File Statistics information for each unique partition tuple is stored as a row in any of the data file format of the table (for example, Parquet or ORC). These rows must be sorted (in ascending manner with NULL FIRST) by `partition` field to optimize filtering rows while scanning. @@ -769,16 +923,18 @@ Note that partition data tuple's schema is based on the partition spec output us The unified partition type is a struct containing all fields that have ever been a part of any spec in the table and sorted by the field ids in ascending order. In other words, the struct fields represent a union of all known partition fields sorted in ascending order by the field ids. + For example, -1) `spec#0` has two fields `{field#1, field#2}` + +1. `spec#0` has two fields `{field#1, field#2}` and then the table has evolved into `spec#1` which has three fields `{field#1, field#2, field#3}`. The unified partition type looks like `Struct`. -2) `spec#0` has two fields `{field#1, field#2}` +2. `spec#0` has two fields `{field#1, field#2}` and then the table has evolved into `spec#1` which has just one field `{field#2}`. The unified partition type looks like `Struct`. -#### Commit Conflict Resolution and Retry +### Commit Conflict Resolution and Retry When two commits happen at the same time and are based on the same version, only one commit will succeed. In most cases, the failed commit can be applied to the new current version of table metadata and retried. Updates verify the conditions under which they can be applied to a new version and retry if those conditions are met. @@ -934,6 +1090,7 @@ Maps with non-string keys must use an array representation with the `map` logica |Type|Avro type|Notes| |--- |--- |--- | +|**`unknown`**|`null` or omitted|| |**`boolean`**|`boolean`|| |**`int`**|`int`|| |**`long`**|`long`|| @@ -987,6 +1144,7 @@ Lists must use the [3-level representation](https://github.com/apache/parquet-fo | Type | Parquet physical type | Logical type | Notes | |--------------------|--------------------------------------------------------------------|---------------------------------------------|----------------------------------------------------------------| +| **`unknown`** | None | | Omit from data files | | **`boolean`** | `boolean` | | | | **`int`** | `int` | | | | **`long`** | `long` | | | @@ -1008,12 +1166,16 @@ Lists must use the [3-level representation](https://github.com/apache/parquet-fo | **`map`** | `3-level map` | `MAP` | See Parquet docs for 3-level representation. | +When reading an `unknown` column, any corresponding column must be ignored and replaced with `null` values. + + ### ORC **Data Type Mappings** | Type | ORC type | ORC type attributes | Notes | |--------------------|---------------------|------------------------------------------------------|-----------------------------------------------------------------------------------------| +| **`unknown`** | None | | Omit from data files | | **`boolean`** | `boolean` | | | | **`int`** | `int` | | ORC `tinyint` and `smallint` would also map to **`int`**. | | **`long`** | `long` | | | @@ -1063,10 +1225,10 @@ The 32-bit hash implementation is 32-bit Murmur3 hash, x86 variant, seeded with | **`time`** | `hashLong(microsecsFromMidnight(v))` | `22:31:08` → `-662762989` | | **`timestamp`** | `hashLong(microsecsFromUnixEpoch(v))` | `2017-11-16T22:31:08` → `-2047944441`
`2017-11-16T22:31:08.000001` → `-1207196810` | | **`timestamptz`** | `hashLong(microsecsFromUnixEpoch(v))` | `2017-11-16T14:31:08-08:00` → `-2047944441`
`2017-11-16T14:31:08.000001-08:00` → `-1207196810` | -| **`timestamp_ns`** | `hashLong(nanosecsFromUnixEpoch(v))` | `2017-11-16T22:31:08` → `-737750069`
`2017-11-16T22:31:08.000001` → `-976603392`
`2017-11-16T22:31:08.000000001` → `-160215926` | -| **`timestamptz_ns`** | `hashLong(nanosecsFromUnixEpoch(v))` | `2017-11-16T14:31:08-08:00` → `-737750069`
`2017-11-16T14:31:08.000001-08:00` → `-976603392`
`2017-11-16T14:31:08.000000001-08:00` → `-160215926` | +| **`timestamp_ns`** | `hashLong(microsecsFromUnixEpoch(v))` [3] | `2017-11-16T22:31:08` → `-2047944441`
`2017-11-16T22:31:08.000001001` → `-1207196810` | +| **`timestamptz_ns`** | `hashLong(microsecsFromUnixEpoch(v))` [3]| `2017-11-16T14:31:08-08:00` → `-2047944441`
`2017-11-16T14:31:08.000001001-08:00` → `-1207196810` | | **`string`** | `hashBytes(utf8Bytes(v))` | `iceberg` → `1210000089` | -| **`uuid`** | `hashBytes(uuidBytes(v))` [3] | `f79c3e09-677c-4bbd-a479-3f349cb785e7` → `1488055340` | +| **`uuid`** | `hashBytes(uuidBytes(v))` [4] | `f79c3e09-677c-4bbd-a479-3f349cb785e7` → `1488055340` | | **`fixed(L)`** | `hashBytes(v)` | `00 01 02 03` → `-188683207` | | **`binary`** | `hashBytes(v)` | `00 01 02 03` → `-188683207` | @@ -1074,18 +1236,20 @@ The types below are not currently valid for bucketing, and so are not hashed. Ho | Primitive type | Hash specification | Test value | |--------------------|-------------------------------------------|--------------------------------------------| +| **`unknown`** | always `null` | | | **`boolean`** | `false: hashInt(0)`, `true: hashInt(1)` | `true` → `1392991556` | -| **`float`** | `hashLong(doubleToLongBits(double(v))` [4]| `1.0F` → `-142385009`, `0.0F` → `1669671676`, `-0.0F` → `1669671676` | -| **`double`** | `hashLong(doubleToLongBits(v))` [4]| `1.0D` → `-142385009`, `0.0D` → `1669671676`, `-0.0D` → `1669671676` | +| **`float`** | `hashLong(doubleToLongBits(double(v))` [5]| `1.0F` → `-142385009`, `0.0F` → `1669671676`, `-0.0F` → `1669671676` | +| **`double`** | `hashLong(doubleToLongBits(v))` [5]| `1.0D` → `-142385009`, `0.0D` → `1669671676`, `-0.0D` → `1669671676` | Notes: 1. Integer and long hash results must be identical for all integer values. This ensures that schema evolution does not change bucket partition values if integer types are promoted. 2. Decimal values are hashed using the minimum number of bytes required to hold the unscaled value as a two’s complement big-endian; this representation does not include padding bytes required for storage in a fixed-length array. Hash results are not dependent on decimal scale, which is part of the type, not the data value. -3. UUIDs are encoded using big endian. The test UUID for the example above is: `f79c3e09-677c-4bbd-a479-3f349cb785e7`. This UUID encoded as a byte array is: +3. Nanosecond timestamps must be converted to microsecond precision before hashing to ensure timestamps have the same hash value. +4. UUIDs are encoded using big endian. The test UUID for the example above is: `f79c3e09-677c-4bbd-a479-3f349cb785e7`. This UUID encoded as a byte array is: `F7 9C 3E 09 67 7C 4B BD A4 79 3F 34 9C B7 85 E7` -4. `doubleToLongBits` must give the IEEE 754 compliant bit representation of the double value. All `NaN` bit patterns must be canonicalized to `0x7ff8000000000000L`. Negative zero (`-0.0`) must be canonicalized to positive zero (`0.0`). Float hash values are the result of hashing the float cast to double to ensure that schema evolution does not change hash values if float types are promoted. +5. `doubleToLongBits` must give the IEEE 754 compliant bit representation of the double value. All `NaN` bit patterns must be canonicalized to `0x7ff8000000000000L`. Negative zero (`-0.0`) must be canonicalized to positive zero (`0.0`). Float hash values are the result of hashing the float cast to double to ensure that schema evolution does not change hash values if float types are promoted. ## Appendix C: JSON serialization @@ -1103,6 +1267,7 @@ Types are serialized according to this table: |Type|JSON representation|Example| |--- |--- |--- | +|**`unknown`**|`JSON string: "unknown"`|`"unknown"`| |**`boolean`**|`JSON string: "boolean"`|`"boolean"`| |**`int`**|`JSON string: "int"`|`"int"`| |**`long`**|`JSON string: "long"`|`"long"`| @@ -1230,7 +1395,7 @@ Name mapping is serialized as a list of field mapping JSON Objects which are ser |Field mapping field|JSON representation|Example| |--- |--- |--- | |**`names`**|`JSON list of strings`|`["latitude", "lat"]`| -|**`field_id`**|`JSON int`|`1`| +|**`field-id`**|`JSON int`|`1`| |**`fields`**|`JSON field mappings (list of objects)`|`[{ `
  `"field-id": 4,`
  `"names": ["latitude", "lat"]`
`}, {`
  `"field-id": 5,`
  `"names": ["longitude", "long"]`
`}]`| Example @@ -1251,6 +1416,7 @@ This serialization scheme is for storing single values as individual binary valu | Type | Binary serialization | |------------------------------|--------------------------------------------------------------------------------------------------------------| +| **`unknown`** | Not supported | | **`boolean`** | `0x00` for false, non-zero byte for true | | **`int`** | Stored as 4-byte little-endian | | **`long`** | Stored as 8-byte little-endian | @@ -1303,12 +1469,13 @@ This serialization scheme is for storing single values as individual binary valu ### Version 3 Default values are added to struct fields in v3. + * The `write-default` is a forward-compatible change because it is only used at write time. Old writers will fail because the field is missing. * Tables with `initial-default` will be read correctly by older readers if `initial-default` is always null for optional fields. Otherwise, old readers will default optional columns with null. Old readers will fail to read required fields which are populated by `initial-default` because that default is not supported. -Types `timestamp_ns` and `timestamptz_ns` are added in v3. +Types `unknown`, `timestamp_ns`, and `timestamptz_ns` are added in v3. -All readers are required to read tables with unknown partition transforms, ignoring them. +All readers are required to read tables with unknown partition transforms, ignoring the unsupported partition fields when filtering. Writing v3 metadata: @@ -1407,3 +1574,4 @@ Iceberg supports two types of histories for tables. A history of previous "curre might indicate different snapshot IDs for a specific timestamp. The discrepancies can be caused by a variety of table operations (e.g. updating the `current-snapshot-id` can be used to set the snapshot of a table to any arbitrary snapshot, which might have a lineage derived from a table branch or no lineage at all). When processing point in time queries implementations should use "snapshot-log" metadata to lookup the table state at the given point in time. This ensures time-travel queries reflect the state of the table at the provided timestamp. For example a SQL query like `SELECT * FROM prod.db.table TIMESTAMP AS OF '1986-10-26 01:21:00Z';` would find the snapshot of the Iceberg table just prior to '1986-10-26 01:21:00 UTC' in the snapshot logs and use the metadata from that snapshot to perform the scan of the table. If no snapshot exists prior to the timestamp given or "snapshot-log" is not populated (it is an optional field), then systems should raise an informative error message about the missing metadata. + diff --git a/gcp-bundle/build.gradle b/gcp-bundle/build.gradle index 1b339a66c195..e7d1b5ddb468 100644 --- a/gcp-bundle/build.gradle +++ b/gcp-bundle/build.gradle @@ -19,7 +19,7 @@ project(":iceberg-gcp-bundle") { - apply plugin: 'io.github.goooler.shadow' + apply plugin: 'com.gradleup.shadow' tasks.jar.dependsOn tasks.shadowJar diff --git a/gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java b/gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java index 4465ee29012a..4f60e2f91f91 100644 --- a/gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java +++ b/gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java @@ -46,6 +46,7 @@ public class GCPProperties implements Serializable { /** Configure the batch size used when deleting multiple files from a given GCS bucket */ public static final String GCS_DELETE_BATCH_SIZE = "gcs.delete.batch-size"; + /** * Max possible batch size for deletion. Currently, a max of 100 keys is advised, so we default to * a number below that. https://cloud.google.com/storage/docs/batch diff --git a/gradle.properties b/gradle.properties index fcbe7d8de012..dc1e1a509b01 100644 --- a/gradle.properties +++ b/gradle.properties @@ -22,6 +22,8 @@ systemProp.defaultHiveVersions=2 systemProp.knownHiveVersions=2,3 systemProp.defaultSparkVersions=3.5 systemProp.knownSparkVersions=3.3,3.4,3.5 +systemProp.defaultKafkaVersions=3 +systemProp.knownKafkaVersions=3 systemProp.defaultScalaVersion=2.12 systemProp.knownScalaVersions=2.12,2.13 # enable the Gradle build cache - speeds up builds! diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 4002623dfa0e..13ce4da5471b 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -24,33 +24,35 @@ activation = "1.1.1" aliyun-sdk-oss = "3.10.2" antlr = "4.9.3" aircompressor = "0.27" +apiguardian = "1.1.2" arrow = "15.0.2" avro = "1.12.0" assertj-core = "3.26.3" awaitility = "4.2.2" -awssdk-bom = "2.27.2" -azuresdk-bom = "1.2.25" -awssdk-s3accessgrants = "2.0.0" +awssdk-bom = "2.29.1" +azuresdk-bom = "1.2.28" +awssdk-s3accessgrants = "2.3.0" caffeine = "2.9.3" calcite = "1.10.0" -datasketches = "6.0.0" -delta-standalone = "3.2.0" -delta-spark = "3.2.0" +datasketches = "6.1.1" +delta-standalone = "3.2.1" +delta-spark = "3.2.1" esotericsoftware-kryo = "4.0.3" -errorprone-annotations = "2.30.0" +errorprone-annotations = "2.35.1" +failsafe = "3.3.2" findbugs-jsr305 = "3.0.2" flink118 = { strictly = "1.18.1"} flink119 = { strictly = "1.19.0"} flink120 = { strictly = "1.20.0"} -google-libraries-bom = "26.44.0" -guava = "33.3.0-jre" +google-libraries-bom = "26.49.0" +guava = "33.3.1-jre" hadoop2 = "2.7.3" hadoop3 = "3.3.6" -httpcomponents-httpclient5 = "5.3.1" +httpcomponents-httpclient5 = "5.4" hive2 = { strictly = "2.3.9"} # see rich version usage explanation above hive3 = "3.1.3" immutables-value = "2.10.1" -jackson-bom = "2.14.2" +jackson-bom = "2.18.0" jackson211 = { strictly = "2.11.4"} # see rich version usage explanation above jackson212 = { strictly = "2.12.3"} jackson213 = { strictly = "2.13.4"} @@ -60,33 +62,31 @@ jakarta-el-api = "3.0.3" jakarta-servlet-api = "6.1.0" jaxb-api = "2.3.1" jaxb-runtime = "2.3.9" -jetty = "11.0.22" -junit = "5.10.1" +jetty = "11.0.24" +junit = "5.11.3" +junit-platform = "1.11.3" kafka = "3.8.0" kryo-shaded = "4.0.3" -microprofile-openapi-api = "3.1.1" +microprofile-openapi-api = "3.1.2" mockito = "4.11.0" mockserver = "5.15.0" -nessie = "0.95.0" -netty-buffer = "4.1.112.Final" -netty-buffer-compat = "4.1.112.Final" +nessie = "0.99.0" +netty-buffer = "4.1.114.Final" +netty-buffer-compat = "4.1.114.Final" object-client-bundle = "3.3.2" orc = "1.9.4" -parquet = "1.13.1" +parquet = "1.14.3" pig = "0.17.0" -roaringbitmap = "1.2.1" -s3mock-junit5 = "2.17.0" +roaringbitmap = "1.3.0" scala-collection-compat = "2.12.0" -slf4j = "1.7.36" -snowflake-jdbc = "3.18.0" +slf4j = "2.0.16" +snowflake-jdbc = "3.19.1" spark-hive33 = "3.3.4" -spark-hive34 = "3.4.3" +spark-hive34 = "3.4.4" spark-hive35 = "3.5.2" -spring-boot = "2.7.18" -spring-web = "5.3.39" -sqlite-jdbc = "3.46.0.1" -testcontainers = "1.20.1" -tez010 = "0.10.3" +sqlite-jdbc = "3.47.0.0" +testcontainers = "1.20.3" +tez010 = "0.10.4" tez08 = { strictly = "0.8.4"} # see rich version usage explanation above [libraries] @@ -107,6 +107,7 @@ calcite-druid = { module = "org.apache.calcite:calcite-druid", version.ref = "ca datasketches = { module = "org.apache.datasketches:datasketches-java", version.ref = "datasketches" } delta-standalone = { module = "io.delta:delta-standalone_2.12", version.ref = "delta-standalone" } errorprone-annotations = { module = "com.google.errorprone:error_prone_annotations", version.ref = "errorprone-annotations" } +failsafe = { module = "dev.failsafe:failsafe", version.ref = "failsafe"} findbugs-jsr305 = { module = "com.google.code.findbugs:jsr305", version.ref = "findbugs-jsr305" } flink118-avro = { module = "org.apache.flink:flink-avro", version.ref = "flink118" } flink118-connector-base = { module = "org.apache.flink:flink-connector-base", version.ref = "flink118" } @@ -176,6 +177,7 @@ slf4j-simple = { module = "org.slf4j:slf4j-simple", version.ref = "slf4j" } snowflake-jdbc = { module = "net.snowflake:snowflake-jdbc", version.ref = "snowflake-jdbc" } # test libraries +apiguardian = { module = "org.apiguardian:apiguardian-api", version.ref = "apiguardian" } assertj-core = { module = "org.assertj:assertj-core", version.ref = "assertj-core" } awaitility = { module = "org.awaitility:awaitility", version.ref = "awaitility" } delta-spark = { module = "io.delta:delta-spark_2.12", version.ref = "delta-spark" } @@ -202,6 +204,8 @@ jetty-server = { module = "org.eclipse.jetty:jetty-server", version.ref = "jetty jetty-servlet = { module = "org.eclipse.jetty:jetty-servlet", version.ref = "jetty" } junit-jupiter = { module = "org.junit.jupiter:junit-jupiter", version.ref = "junit" } junit-jupiter-engine = { module = "org.junit.jupiter:junit-jupiter-engine", version.ref = "junit" } +junit-suite-api = { module = "org.junit.platform:junit-platform-suite-api", version.ref = "junit-platform" } +junit-suite-engine = { module = "org.junit.platform:junit-platform-suite-engine", version.ref = "junit-platform" } junit-vintage-engine = { module = "org.junit.vintage:junit-vintage-engine", version.ref = "junit" } kryo-shaded = { module = "com.esotericsoftware:kryo-shaded", version.ref = "kryo-shaded" } mockito-core = { module = "org.mockito:mockito-core", version.ref = "mockito" } @@ -213,12 +217,10 @@ nessie-jaxrs-testextension = { module = "org.projectnessie.nessie:nessie-jaxrs-t nessie-versioned-storage-inmemory-tests = { module = "org.projectnessie.nessie:nessie-versioned-storage-inmemory-tests", version.ref = "nessie" } nessie-versioned-storage-testextension = { module = "org.projectnessie.nessie:nessie-versioned-storage-testextension", version.ref = "nessie" } orc-tools = { module = "org.apache.orc:orc-tools", version.ref = "orc" } -s3mock-junit5 = { module = "com.adobe.testing:s3mock-junit5", version.ref = "s3mock-junit5" } -spring-boot-starter-jetty = { module = "org.springframework.boot:spring-boot-starter-jetty", version.ref = "spring-boot" } -spring-boot-starter-web = { module = "org.springframework.boot:spring-boot-starter-web", version.ref = "spring-boot" } -spring-web = { module = "org.springframework:spring-web", version.ref = "spring-web" } sqlite-jdbc = { module = "org.xerial:sqlite-jdbc", version.ref = "sqlite-jdbc" } testcontainers = { module = "org.testcontainers:testcontainers", version.ref = "testcontainers" } +testcontainers-junit-jupiter = { module = "org.testcontainers:junit-jupiter", version.ref = "testcontainers" } +testcontainers-minio = { module = "org.testcontainers:minio", version.ref = "testcontainers" } tez010-dag = { module = "org.apache.tez:tez-dag", version.ref = "tez010" } tez010-mapreduce = { module = "org.apache.tez:tez-mapreduce", version.ref = "tez010" } tez08-dag = { module = "org.apache.tez:tez-dag", version.ref = "tez08" } diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties index 68e8816d71c9..fb602ee2af06 100644 --- a/gradle/wrapper/gradle-wrapper.properties +++ b/gradle/wrapper/gradle-wrapper.properties @@ -1,7 +1,7 @@ distributionBase=GRADLE_USER_HOME distributionPath=wrapper/dists -distributionSha256Sum=d725d707bfabd4dfdc958c624003b3c80accc03f7037b5122c4b1d0ef15cecab -distributionUrl=https\://services.gradle.org/distributions/gradle-8.9-bin.zip +distributionSha256Sum=31c55713e40233a8303827ceb42ca48a47267a0ad4bab9177123121e71524c26 +distributionUrl=https\://services.gradle.org/distributions/gradle-8.10.2-bin.zip networkTimeout=10000 validateDistributionUrl=true zipStoreBase=GRADLE_USER_HOME diff --git a/gradlew b/gradlew index f7166e1f63af..57ebc02755ab 100755 --- a/gradlew +++ b/gradlew @@ -87,7 +87,7 @@ APP_BASE_NAME=${0##*/} APP_HOME=$( cd "${APP_HOME:-./}" > /dev/null && pwd -P ) || exit if [ ! -e $APP_HOME/gradle/wrapper/gradle-wrapper.jar ]; then - curl -o $APP_HOME/gradle/wrapper/gradle-wrapper.jar https://raw.githubusercontent.com/gradle/gradle/v8.9.0/gradle/wrapper/gradle-wrapper.jar + curl -o $APP_HOME/gradle/wrapper/gradle-wrapper.jar https://raw.githubusercontent.com/gradle/gradle/v8.10.2/gradle/wrapper/gradle-wrapper.jar fi # Use the maximum available, or set MAX_FD != -1 to use that value. diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java index 5c58222f0c01..1cf738d736cb 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java @@ -27,6 +27,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.TableType; import org.apache.hadoop.hive.metastore.api.AlreadyExistsException; import org.apache.hadoop.hive.metastore.api.Database; import org.apache.hadoop.hive.metastore.api.InvalidOperationException; @@ -34,19 +35,21 @@ import org.apache.hadoop.hive.metastore.api.PrincipalType; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.api.UnknownDBException; -import org.apache.iceberg.BaseMetastoreCatalog; import org.apache.iceberg.BaseMetastoreTableOperations; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.ClientPool; +import org.apache.iceberg.Schema; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableOperations; +import org.apache.iceberg.Transaction; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.SupportsNamespaces; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.exceptions.NamespaceNotEmptyException; import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.exceptions.NoSuchViewException; import org.apache.iceberg.exceptions.NotFoundException; import org.apache.iceberg.hadoop.HadoopFileIO; import org.apache.iceberg.io.FileIO; @@ -56,13 +59,21 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.LocationUtil; +import org.apache.iceberg.view.BaseMetastoreViewCatalog; +import org.apache.iceberg.view.View; +import org.apache.iceberg.view.ViewBuilder; +import org.apache.iceberg.view.ViewMetadata; +import org.apache.iceberg.view.ViewOperations; import org.apache.thrift.TException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class HiveCatalog extends BaseMetastoreCatalog implements SupportsNamespaces, Configurable { +public class HiveCatalog extends BaseMetastoreViewCatalog + implements SupportsNamespaces, Configurable { public static final String LIST_ALL_TABLES = "list-all-tables"; public static final String LIST_ALL_TABLES_DEFAULT = "false"; @@ -117,6 +128,16 @@ public void initialize(String inputName, Map properties) { this.fileIOTracker = new FileIOTracker(); } + @Override + public TableBuilder buildTable(TableIdentifier identifier, Schema schema) { + return new ViewAwareTableBuilder(identifier, schema); + } + + @Override + public ViewBuilder buildView(TableIdentifier identifier) { + return new TableAwareViewBuilder(identifier); + } + @Override public List listTables(Namespace namespace) { Preconditions.checkArgument( @@ -156,6 +177,38 @@ public List listTables(Namespace namespace) { } } + @Override + public List listViews(Namespace namespace) { + Preconditions.checkArgument( + isValidateNamespace(namespace), "Missing database in namespace: %s", namespace); + + try { + String database = namespace.level(0); + List viewNames = + clients.run(client -> client.getTables(database, "*", TableType.VIRTUAL_VIEW)); + + // Retrieving the Table objects from HMS in batches to avoid OOM + List filteredTableIdentifiers = Lists.newArrayList(); + Iterable> viewNameSets = Iterables.partition(viewNames, 100); + + for (List viewNameSet : viewNameSets) { + filteredTableIdentifiers.addAll( + listIcebergTables(viewNameSet, namespace, HiveOperationsBase.ICEBERG_VIEW_TYPE_VALUE)); + } + + return filteredTableIdentifiers; + } catch (UnknownDBException e) { + throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace); + + } catch (TException e) { + throw new RuntimeException("Failed to list all views under namespace " + namespace, e); + + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted in call to listViews", e); + } + } + @Override public String name() { return name; @@ -213,11 +266,57 @@ public boolean dropTable(TableIdentifier identifier, boolean purge) { } } + @Override + public boolean dropView(TableIdentifier identifier) { + if (!isValidIdentifier(identifier)) { + return false; + } + + try { + String database = identifier.namespace().level(0); + String viewName = identifier.name(); + + HiveViewOperations ops = (HiveViewOperations) newViewOps(identifier); + ViewMetadata lastViewMetadata = null; + try { + lastViewMetadata = ops.current(); + } catch (NotFoundException e) { + LOG.warn("Failed to load view metadata for view: {}", identifier, e); + } + + clients.run( + client -> { + client.dropTable(database, viewName, false, false); + return null; + }); + + if (lastViewMetadata != null) { + CatalogUtil.dropViewMetadata(ops.io(), lastViewMetadata); + } + + LOG.info("Dropped view: {}", identifier); + return true; + } catch (NoSuchObjectException e) { + LOG.info("Skipping drop, view does not exist: {}", identifier, e); + return false; + } catch (TException e) { + throw new RuntimeException("Failed to drop view " + identifier, e); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted in call to dropView", e); + } + } + @Override public void renameTable(TableIdentifier from, TableIdentifier originalTo) { renameTableOrView(from, originalTo, HiveOperationsBase.ContentType.TABLE); } + @Override + public void renameView(TableIdentifier from, TableIdentifier to) { + renameTableOrView(from, to, HiveOperationsBase.ContentType.VIEW); + } + private List listIcebergTables( List tableNames, Namespace namespace, String tableTypeProp) throws TException, InterruptedException { @@ -233,13 +332,12 @@ private List listIcebergTables( .collect(Collectors.toList()); } + @SuppressWarnings("checkstyle:CyclomaticComplexity") private void renameTableOrView( TableIdentifier from, TableIdentifier originalTo, HiveOperationsBase.ContentType contentType) { - if (!isValidIdentifier(from)) { - throw new NoSuchTableException("Invalid identifier: %s", from); - } + Preconditions.checkArgument(isValidIdentifier(from), "Invalid identifier: %s", from); TableIdentifier to = removeCatalogName(originalTo); Preconditions.checkArgument(isValidIdentifier(to), "Invalid identifier: %s", to); @@ -248,6 +346,16 @@ private void renameTableOrView( "Cannot rename %s to %s. Namespace does not exist: %s", from, to, to.namespace()); } + if (tableExists(to)) { + throw new org.apache.iceberg.exceptions.AlreadyExistsException( + "Cannot rename %s to %s. Table already exists", from, to); + } + + if (viewExists(to)) { + throw new org.apache.iceberg.exceptions.AlreadyExistsException( + "Cannot rename %s to %s. View already exists", from, to); + } + String toDatabase = to.namespace().level(0); String fromDatabase = from.namespace().level(0); String fromName = from.name(); @@ -268,7 +376,12 @@ private void renameTableOrView( LOG.info("Renamed {} from {}, to {}", contentType.value(), from, to); } catch (NoSuchObjectException e) { - throw new NoSuchTableException("Table does not exist: %s", from); + switch (contentType) { + case TABLE: + throw new NoSuchTableException("Cannot rename %s to %s. Table does not exist", from, to); + case VIEW: + throw new NoSuchViewException("Cannot rename %s to %s. View does not exist", from, to); + } } catch (InvalidOperationException e) { if (e.getMessage() != null @@ -295,7 +408,7 @@ private void validateTableIsIcebergTableOrView( HiveOperationsBase.validateTableIsIceberg(table, fullName); break; case VIEW: - throw new UnsupportedOperationException("View is not supported."); + HiveOperationsBase.validateTableIsIcebergView(table, fullName); } } @@ -522,6 +635,11 @@ public TableOperations newTableOps(TableIdentifier tableIdentifier) { return ops; } + @Override + protected ViewOperations newViewOps(TableIdentifier identifier) { + return new HiveViewOperations(conf, clients, fileIO, name, identifier); + } + @Override protected String defaultWarehouseLocation(TableIdentifier tableIdentifier) { // This is a little edgy since we basically duplicate the HMS location generation logic. @@ -660,4 +778,72 @@ void setListAllTables(boolean listAllTables) { ClientPool clientPool() { return clients; } + + /** + * The purpose of this class is to add view detection only for Hive-Specific tables. Hive catalog + * follows checks at different levels: 1. During refresh, it validates if the table is an iceberg + * table or not. 2. During commit, it validates if there is any concurrent commit with table or + * table-name already exists. This class helps to do the validation on an early basis. + */ + private class ViewAwareTableBuilder extends BaseMetastoreViewCatalogTableBuilder { + + private final TableIdentifier identifier; + + private ViewAwareTableBuilder(TableIdentifier identifier, Schema schema) { + super(identifier, schema); + this.identifier = identifier; + } + + @Override + public Transaction createOrReplaceTransaction() { + if (viewExists(identifier)) { + throw new org.apache.iceberg.exceptions.AlreadyExistsException( + "View with same name already exists: %s", identifier); + } + return super.createOrReplaceTransaction(); + } + + @Override + public org.apache.iceberg.Table create() { + if (viewExists(identifier)) { + throw new org.apache.iceberg.exceptions.AlreadyExistsException( + "View with same name already exists: %s", identifier); + } + return super.create(); + } + } + + /** + * The purpose of this class is to add table detection only for Hive-Specific view. Hive catalog + * follows checks at different levels: 1. During refresh, it validates if the view is an iceberg + * view or not. 2. During commit, it validates if there is any concurrent commit with view or + * view-name already exists. This class helps to do the validation on an early basis. + */ + private class TableAwareViewBuilder extends BaseViewBuilder { + + private final TableIdentifier identifier; + + private TableAwareViewBuilder(TableIdentifier identifier) { + super(identifier); + this.identifier = identifier; + } + + @Override + public View createOrReplace() { + if (tableExists(identifier)) { + throw new org.apache.iceberg.exceptions.AlreadyExistsException( + "Table with same name already exists: %s", identifier); + } + return super.createOrReplace(); + } + + @Override + public View create() { + if (tableExists(identifier)) { + throw new org.apache.iceberg.exceptions.AlreadyExistsException( + "Table with same name already exists: %s", identifier); + } + return super.create(); + } + } } diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveOperationsBase.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveOperationsBase.java index 6500e724a4f0..4c78c43096fe 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveOperationsBase.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveOperationsBase.java @@ -33,6 +33,7 @@ import org.apache.iceberg.SchemaParser; import org.apache.iceberg.TableProperties; import org.apache.iceberg.exceptions.NoSuchIcebergTableException; +import org.apache.iceberg.exceptions.NoSuchIcebergViewException; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -53,6 +54,7 @@ interface HiveOperationsBase { long HIVE_TABLE_PROPERTY_MAX_SIZE_DEFAULT = 32672; String NO_LOCK_EXPECTED_KEY = "expected_parameter_key"; String NO_LOCK_EXPECTED_VALUE = "expected_parameter_value"; + String ICEBERG_VIEW_TYPE_VALUE = "iceberg-view"; enum ContentType { TABLE("Table"), @@ -129,6 +131,17 @@ static void validateTableIsIceberg(Table table, String fullName) { tableType); } + static void validateTableIsIcebergView(Table table, String fullName) { + String tableTypeProp = table.getParameters().get(BaseMetastoreTableOperations.TABLE_TYPE_PROP); + NoSuchIcebergViewException.check( + TableType.VIRTUAL_VIEW.name().equalsIgnoreCase(table.getTableType()) + && ICEBERG_VIEW_TYPE_VALUE.equalsIgnoreCase(tableTypeProp), + "Not an iceberg view: %s (type=%s) (tableType=%s)", + fullName, + tableTypeProp, + table.getTableType()); + } + default void persistTable(Table hmsTable, boolean updateHiveTable, String metadataLocation) throws TException, InterruptedException { if (updateHiveTable) { diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java index 64f091385297..518daaf6acd1 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java @@ -167,7 +167,7 @@ protected void doRefresh() { refreshFromMetadataLocation(metadataLocation, metadataRefreshMaxRetries); } - @SuppressWarnings("checkstyle:CyclomaticComplexity") + @SuppressWarnings({"checkstyle:CyclomaticComplexity", "MethodLength"}) @Override protected void doCommit(TableMetadata base, TableMetadata metadata) { boolean newTable = base == null; @@ -191,6 +191,10 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) { if (newTable && tbl.getParameters().get(BaseMetastoreTableOperations.METADATA_LOCATION_PROP) != null) { + if (TableType.VIRTUAL_VIEW.name().equalsIgnoreCase(tbl.getTableType())) { + throw new AlreadyExistsException( + "View with same name already exists: %s.%s", database, tableName); + } throw new AlreadyExistsException("Table already exists: %s.%s", database, tableName); } diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveViewOperations.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveViewOperations.java new file mode 100644 index 000000000000..4fc71299d457 --- /dev/null +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveViewOperations.java @@ -0,0 +1,389 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.hive; + +import static java.util.Collections.emptySet; + +import java.util.Collections; +import java.util.Locale; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.TableType; +import org.apache.hadoop.hive.metastore.api.InvalidObjectException; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.iceberg.BaseMetastoreTableOperations; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.ClientPool; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.CommitStateUnknownException; +import org.apache.iceberg.exceptions.NoSuchViewException; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.hadoop.ConfigProperties; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.view.BaseViewOperations; +import org.apache.iceberg.view.SQLViewRepresentation; +import org.apache.iceberg.view.ViewMetadata; +import org.apache.iceberg.view.ViewRepresentation; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Hive implementation of Iceberg {@link org.apache.iceberg.view.ViewOperations}. */ +final class HiveViewOperations extends BaseViewOperations implements HiveOperationsBase { + private static final Logger LOG = LoggerFactory.getLogger(HiveViewOperations.class); + + private final String fullName; + private final String database; + private final String viewName; + private final FileIO fileIO; + private final ClientPool metaClients; + private final long maxHiveTablePropertySize; + private final Configuration conf; + private final String catalogName; + + HiveViewOperations( + Configuration conf, + ClientPool metaClients, + FileIO fileIO, + String catalogName, + TableIdentifier viewIdentifier) { + this.conf = conf; + this.catalogName = catalogName; + this.metaClients = metaClients; + this.fileIO = fileIO; + this.fullName = CatalogUtil.fullTableName(catalogName, viewIdentifier); + this.database = viewIdentifier.namespace().level(0); + this.viewName = viewIdentifier.name(); + this.maxHiveTablePropertySize = + conf.getLong(HIVE_TABLE_PROPERTY_MAX_SIZE, HIVE_TABLE_PROPERTY_MAX_SIZE_DEFAULT); + } + + @Override + public void doRefresh() { + String metadataLocation = null; + Table table; + + try { + table = metaClients.run(client -> client.getTable(database, viewName)); + HiveOperationsBase.validateTableIsIcebergView(table, fullName); + + metadataLocation = + table.getParameters().get(BaseMetastoreTableOperations.METADATA_LOCATION_PROP); + + } catch (NoSuchObjectException e) { + if (currentMetadataLocation() != null) { + throw new NoSuchViewException("View does not exist: %s.%s", database, viewName); + } + } catch (TException e) { + String errMsg = + String.format("Failed to get view info from metastore %s.%s", database, viewName); + throw new RuntimeException(errMsg, e); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted during refresh", e); + } + + refreshFromMetadataLocation(metadataLocation); + } + + @SuppressWarnings("checkstyle:CyclomaticComplexity") + @Override + public void doCommit(ViewMetadata base, ViewMetadata metadata) { + boolean newView = base == null; + String newMetadataLocation = writeNewMetadataIfRequired(metadata); + boolean hiveEngineEnabled = false; + + CommitStatus commitStatus = CommitStatus.FAILURE; + boolean updateHiveView = false; + + HiveLock lock = lockObject(); + try { + lock.lock(); + + Table tbl = loadHmsTable(); + + if (tbl != null) { + // If we try to create the view but the metadata location is already set, then we had a + // concurrent commit + if (newView + && tbl.getParameters().get(BaseMetastoreTableOperations.METADATA_LOCATION_PROP) + != null) { + throw new AlreadyExistsException( + "%s already exists: %s.%s", + TableType.VIRTUAL_VIEW.name().equalsIgnoreCase(tbl.getTableType()) + ? ContentType.VIEW.value() + : ContentType.TABLE.value(), + database, + viewName); + } + + updateHiveView = true; + LOG.debug("Committing existing view: {}", fullName); + } else { + tbl = newHMSView(metadata); + LOG.debug("Committing new view: {}", fullName); + } + + tbl.setSd( + HiveOperationsBase.storageDescriptor( + metadata.schema(), + metadata.location(), + hiveEngineEnabled)); // set to pick up any schema changes + + String metadataLocation = + tbl.getParameters().get(BaseMetastoreTableOperations.METADATA_LOCATION_PROP); + String baseMetadataLocation = base != null ? base.metadataFileLocation() : null; + if (!Objects.equals(baseMetadataLocation, metadataLocation)) { + throw new CommitFailedException( + "Cannot commit: Base metadata location '%s' is not same as the current view metadata location '%s' for %s.%s", + baseMetadataLocation, metadataLocation, database, viewName); + } + + // get Iceberg props that have been removed + Set removedProps = emptySet(); + if (base != null) { + removedProps = + base.properties().keySet().stream() + .filter(key -> !metadata.properties().containsKey(key)) + .collect(Collectors.toSet()); + } + + setHmsTableParameters(newMetadataLocation, tbl, metadata, removedProps); + + lock.ensureActive(); + + try { + persistTable(tbl, updateHiveView, hiveLockEnabled(conf) ? null : baseMetadataLocation); + lock.ensureActive(); + + commitStatus = CommitStatus.SUCCESS; + } catch (LockException le) { + commitStatus = CommitStatus.UNKNOWN; + throw new CommitStateUnknownException( + "Failed to heartbeat for hive lock while " + + "committing changes. This can lead to a concurrent commit attempt be able to overwrite this commit. " + + "Please check the commit history. If you are running into this issue, try reducing " + + "iceberg.hive.lock-heartbeat-interval-ms.", + le); + } catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException e) { + throw new AlreadyExistsException(e, "View already exists: %s.%s", database, viewName); + + } catch (InvalidObjectException e) { + throw new ValidationException(e, "Invalid Hive object for %s.%s", database, viewName); + + } catch (CommitFailedException | CommitStateUnknownException e) { + throw e; + + } catch (Throwable e) { + if (e.getMessage() != null + && e.getMessage() + .contains( + "The table has been modified. The parameter value for key '" + + BaseMetastoreTableOperations.METADATA_LOCATION_PROP + + "' is")) { + throw new CommitFailedException( + e, "The view %s.%s has been modified concurrently", database, viewName); + } + + if (e.getMessage() != null + && e.getMessage().contains("Table/View 'HIVE_LOCKS' does not exist")) { + throw new RuntimeException( + "Failed to acquire locks from metastore because the underlying metastore " + + "view 'HIVE_LOCKS' does not exist. This can occur when using an embedded metastore which does not " + + "support transactions. To fix this use an alternative metastore.", + e); + } + + LOG.error( + "Cannot tell if commit to {}.{} succeeded, attempting to reconnect and check.", + database, + viewName, + e); + commitStatus = + checkCommitStatus( + viewName, + newMetadataLocation, + metadata.properties(), + () -> checkCurrentMetadataLocation(newMetadataLocation)); + switch (commitStatus) { + case SUCCESS: + break; + case FAILURE: + throw e; + case UNKNOWN: + throw new CommitStateUnknownException(e); + } + } + } catch (TException e) { + throw new RuntimeException( + String.format("Metastore operation failed for %s.%s", database, viewName), e); + + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted during commit", e); + + } catch (LockException e) { + throw new CommitFailedException(e); + + } finally { + HiveOperationsBase.cleanupMetadataAndUnlock(io(), commitStatus, newMetadataLocation, lock); + } + + LOG.info( + "Committed to view {} with the new metadata location {}", fullName, newMetadataLocation); + } + + /** + * Validate if the new metadata location is the current metadata location. + * + * @param newMetadataLocation newly written metadata location + * @return true if the new metadata location is the current metadata location + */ + private boolean checkCurrentMetadataLocation(String newMetadataLocation) { + ViewMetadata metadata = refresh(); + return newMetadataLocation.equals(metadata.metadataFileLocation()); + } + + private void setHmsTableParameters( + String newMetadataLocation, Table tbl, ViewMetadata metadata, Set obsoleteProps) { + Map parameters = + Optional.ofNullable(tbl.getParameters()).orElseGet(Maps::newHashMap); + + // push all Iceberg view properties into HMS + metadata.properties().entrySet().stream() + .filter(entry -> !entry.getKey().equalsIgnoreCase(HiveCatalog.HMS_TABLE_OWNER)) + .forEach(entry -> parameters.put(entry.getKey(), entry.getValue())); + if (metadata.uuid() != null) { + parameters.put("uuid", metadata.uuid()); + } + + // remove any props from HMS that are no longer present in Iceberg view props + obsoleteProps.forEach(parameters::remove); + + parameters.put( + BaseMetastoreTableOperations.TABLE_TYPE_PROP, + ICEBERG_VIEW_TYPE_VALUE.toUpperCase(Locale.ENGLISH)); + parameters.put(BaseMetastoreTableOperations.METADATA_LOCATION_PROP, newMetadataLocation); + + if (currentMetadataLocation() != null && !currentMetadataLocation().isEmpty()) { + parameters.put( + BaseMetastoreTableOperations.PREVIOUS_METADATA_LOCATION_PROP, currentMetadataLocation()); + } + + setSchema(metadata.schema(), parameters); + tbl.setParameters(parameters); + } + + private static boolean hiveLockEnabled(Configuration conf) { + return conf.getBoolean(ConfigProperties.LOCK_HIVE_ENABLED, true); + } + + private Table newHMSView(ViewMetadata metadata) { + final long currentTimeMillis = System.currentTimeMillis(); + String hmsTableOwner = + PropertyUtil.propertyAsString( + metadata.properties(), HiveCatalog.HMS_TABLE_OWNER, HiveHadoopUtil.currentUser()); + String sqlQuery = sqlFor(metadata); + + return new Table( + table(), + database(), + hmsTableOwner, + (int) currentTimeMillis / 1000, + (int) currentTimeMillis / 1000, + Integer.MAX_VALUE, + null, + Collections.emptyList(), + Maps.newHashMap(), + sqlQuery, + sqlQuery, + tableType().name()); + } + + private String sqlFor(ViewMetadata metadata) { + SQLViewRepresentation closest = null; + for (ViewRepresentation representation : metadata.currentVersion().representations()) { + if (representation instanceof SQLViewRepresentation) { + SQLViewRepresentation sqlViewRepresentation = (SQLViewRepresentation) representation; + if (sqlViewRepresentation.dialect().equalsIgnoreCase("hive")) { + return sqlViewRepresentation.sql(); + } else if (closest == null) { + closest = sqlViewRepresentation; + } + } + } + + return closest == null ? null : closest.sql(); + } + + @VisibleForTesting + HiveLock lockObject() { + if (hiveLockEnabled(conf)) { + return new MetastoreLock(conf, metaClients, catalogName, database, viewName); + } else { + return new NoLock(); + } + } + + @Override + protected String viewName() { + return fullName; + } + + @Override + public TableType tableType() { + return TableType.VIRTUAL_VIEW; + } + + @Override + public ClientPool metaClients() { + return metaClients; + } + + @Override + public long maxHiveTablePropertySize() { + return maxHiveTablePropertySize; + } + + @Override + public String database() { + return database; + } + + @Override + public String table() { + return viewName; + } + + @Override + public FileIO io() { + return fileIO; + } +} diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableTest.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableTest.java index 9ae3c97db47c..13c459128dec 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableTest.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableTest.java @@ -213,7 +213,7 @@ public void testDropTable() throws IOException { table.newAppend().appendFile(file1).appendFile(file2).commit(); // delete file2 - table.newDelete().deleteFile(file2.path()).commit(); + table.newDelete().deleteFile(file2).commit(); String manifestListLocation = table.currentSnapshot().manifestListLocation().replace("file:", ""); diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java index 9249deb7598e..7d0eb641a385 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java @@ -155,6 +155,21 @@ private Schema getTestSchema() { required(2, "data", Types.StringType.get())); } + @Test + public void testInvalidIdentifiersWithRename() { + TableIdentifier invalidFrom = TableIdentifier.of(Namespace.of("l1", "l2"), "table1"); + TableIdentifier validTo = TableIdentifier.of(Namespace.of("l1"), "renamedTable"); + assertThatThrownBy(() -> catalog.renameTable(invalidFrom, validTo)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid identifier: " + invalidFrom); + + TableIdentifier validFrom = TableIdentifier.of(Namespace.of("l1"), "table1"); + TableIdentifier invalidTo = TableIdentifier.of(Namespace.of("l1", "l2"), "renamedTable"); + assertThatThrownBy(() -> catalog.renameTable(validFrom, invalidTo)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid identifier: " + invalidTo); + } + @Test public void testCreateTableBuilder() throws Exception { Schema schema = getTestSchema(); diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveViewCatalog.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveViewCatalog.java new file mode 100644 index 000000000000..3c195e256520 --- /dev/null +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveViewCatalog.java @@ -0,0 +1,323 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.hive; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.nio.file.Files; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.TableType; +import org.apache.hadoop.hive.metastore.api.SerDeInfo; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.serde.serdeConstants; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NoSuchIcebergViewException; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.view.BaseView; +import org.apache.iceberg.view.ViewCatalogTests; +import org.apache.thrift.TException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +public class TestHiveViewCatalog extends ViewCatalogTests { + + private HiveCatalog catalog; + + @RegisterExtension + private static final HiveMetastoreExtension HIVE_METASTORE_EXTENSION = + HiveMetastoreExtension.builder().build(); + + @BeforeEach + public void before() throws TException { + catalog = + (HiveCatalog) + CatalogUtil.loadCatalog( + HiveCatalog.class.getName(), + CatalogUtil.ICEBERG_CATALOG_TYPE_HIVE, + ImmutableMap.of( + CatalogProperties.CLIENT_POOL_CACHE_EVICTION_INTERVAL_MS, + String.valueOf(TimeUnit.SECONDS.toMillis(10))), + HIVE_METASTORE_EXTENSION.hiveConf()); + } + + @AfterEach + public void cleanup() throws Exception { + HIVE_METASTORE_EXTENSION.metastore().reset(); + } + + @Override + protected HiveCatalog catalog() { + return catalog; + } + + @Override + protected Catalog tableCatalog() { + return catalog; + } + + @Override + protected boolean requiresNamespaceCreate() { + return true; + } + + @Test + public void testHiveViewAndIcebergViewWithSameName() throws TException, IOException { + String dbName = "hivedb"; + Namespace ns = Namespace.of(dbName); + String viewName = "test_hive_view"; + TableIdentifier identifier = TableIdentifier.of(ns, viewName); + + if (requiresNamespaceCreate()) { + catalog.createNamespace(identifier.namespace()); + } + + assertThat(catalog.listViews(ns)).isEmpty(); + // create a hive table + Table hiveTable = + createHiveView( + viewName, dbName, Files.createTempDirectory("hive-view-tests-name").toString()); + HIVE_METASTORE_EXTENSION.metastoreClient().createTable(hiveTable); + + catalog.setListAllTables(true); + assertThat(catalog.listTables(ns)).containsExactly(identifier).hasSize(1); + + assertThat(catalog.viewExists(identifier)).isFalse(); + + assertThatThrownBy( + () -> + catalog + .buildView(identifier) + .withSchema(SCHEMA) + .withDefaultNamespace(ns) + .withQuery("hive", "select * from hivedb.tbl") + .create()) + .isInstanceOf(NoSuchIcebergViewException.class) + .hasMessageStartingWith("Not an iceberg view: hive.hivedb.test_hive_view"); + } + + @Test + public void testListViewWithHiveView() throws TException, IOException { + String dbName = "hivedb"; + Namespace ns = Namespace.of(dbName); + TableIdentifier identifier = TableIdentifier.of(ns, "test_iceberg_view"); + + if (requiresNamespaceCreate()) { + catalog.createNamespace(identifier.namespace()); + } + + assertThat(catalog.viewExists(identifier)).isFalse(); + assertThat(catalog.listViews(ns)).isEmpty(); + + String hiveViewName = "test_hive_view"; + // create a hive table + Table hiveTable = + createHiveView( + hiveViewName, dbName, Files.createTempDirectory("hive-view-tests-list").toString()); + HIVE_METASTORE_EXTENSION.metastoreClient().createTable(hiveTable); + + catalog.setListAllTables(true); + + assertThat(catalog.listTables(ns)) + .containsExactly(TableIdentifier.of(ns, hiveViewName)) + .hasSize(1); + + assertThat(catalog.listViews(ns)).hasSize(0); + + catalog + .buildView(identifier) + .withSchema(SCHEMA) + .withDefaultNamespace(ns) + .withQuery("hive", "select * from hivedb.tbl") + .create(); + assertThat(catalog.viewExists(identifier)).isTrue(); + + assertThat(catalog.listViews(ns)).containsExactly(identifier).hasSize(1); + } + + @Test + public void testViewWithHiveParameters() throws TException, IOException { + String dbName = "hivedb"; + Namespace ns = Namespace.of(dbName); + TableIdentifier identifier = TableIdentifier.of(ns, "test_iceberg_view"); + + if (requiresNamespaceCreate()) { + catalog.createNamespace(identifier.namespace()); + } + + assertThat(catalog.viewExists(identifier)).isFalse(); + String tableQuery = "select * from hivedb.tbl"; + + catalog + .buildView(identifier) + .withSchema(SCHEMA) + .withDefaultNamespace(ns) + .withQuery("hive", tableQuery) + .create(); + assertThat(catalog.viewExists(identifier)).isTrue(); + + Table hiveTable = + HIVE_METASTORE_EXTENSION.metastoreClient().getTable(dbName, identifier.name()); + assertThat(hiveTable.getViewOriginalText()).isEqualTo(tableQuery); + assertThat(hiveTable.getViewExpandedText()).isEqualTo(tableQuery); + } + + @Test + public void testInvalidIdentifiersWithRename() { + TableIdentifier invalidFrom = TableIdentifier.of(Namespace.of("l1", "l2"), "view"); + TableIdentifier validTo = TableIdentifier.of(Namespace.of("l1"), "renamedView"); + assertThatThrownBy(() -> catalog.renameView(invalidFrom, validTo)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid identifier: " + invalidFrom); + + TableIdentifier validFrom = TableIdentifier.of(Namespace.of("l1"), "view"); + TableIdentifier invalidTo = TableIdentifier.of(Namespace.of("l1", "l2"), "renamedView"); + assertThatThrownBy(() -> catalog.renameView(validFrom, invalidTo)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid identifier: " + invalidTo); + } + + @Test + public void dropViewShouldNotDropMetadataFileIfGcNotEnabled() throws IOException { + String dbName = "hivedb"; + Namespace ns = Namespace.of(dbName); + TableIdentifier identifier = TableIdentifier.of(ns, "test_iceberg_drop_view_gc_disabled"); + if (requiresNamespaceCreate()) { + catalog.createNamespace(identifier.namespace()); + } + + BaseView view = + (BaseView) + catalog + .buildView(identifier) + .withSchema(SCHEMA) + .withDefaultNamespace(ns) + .withQuery("hive", "select * from hivedb.tbl") + .withProperty(TableProperties.GC_ENABLED, "false") + .create(); + + assertThat(catalog.viewExists(identifier)).isTrue(); + + Path viewLocation = new Path(view.location()); + String currentMetadataLocation = view.operations().current().metadataFileLocation(); + + catalog.dropView(identifier); + + assertThat( + viewLocation + .getFileSystem(HIVE_METASTORE_EXTENSION.hiveConf()) + .exists(new Path(currentMetadataLocation))) + .isTrue(); + assertThat(catalog.viewExists(identifier)).isFalse(); + } + + @Test + public void dropViewShouldDropMetadataFileIfGcEnabled() throws IOException { + String dbName = "hivedb"; + Namespace ns = Namespace.of(dbName); + TableIdentifier identifier = TableIdentifier.of(ns, "test_iceberg_drop_view_gc_enabled"); + if (requiresNamespaceCreate()) { + catalog.createNamespace(identifier.namespace()); + } + + BaseView view = + (BaseView) + catalog + .buildView(identifier) + .withSchema(SCHEMA) + .withDefaultNamespace(ns) + .withQuery("hive", "select * from hivedb.tbl") + .withProperty(TableProperties.GC_ENABLED, "true") + .create(); + + assertThat(catalog.viewExists(identifier)).isTrue(); + + Path viewLocation = new Path(view.location()); + String currentMetadataLocation = view.operations().current().metadataFileLocation(); + + assertThat( + viewLocation + .getFileSystem(HIVE_METASTORE_EXTENSION.hiveConf()) + .exists(new Path(currentMetadataLocation))) + .isTrue(); + + catalog.dropView(identifier); + + assertThat( + viewLocation + .getFileSystem(HIVE_METASTORE_EXTENSION.hiveConf()) + .exists(new Path(currentMetadataLocation))) + .isFalse(); + assertThat(catalog.viewExists(identifier)).isFalse(); + } + + private Table createHiveView(String hiveViewName, String dbName, String location) { + Map parameters = Maps.newHashMap(); + parameters.put( + serdeConstants.SERIALIZATION_CLASS, "org.apache.hadoop.hive.serde2.thrift.test.IntString"); + parameters.put( + serdeConstants.SERIALIZATION_FORMAT, "org.apache.thrift.protocol.TBinaryProtocol"); + + SerDeInfo serDeInfo = + new SerDeInfo(null, "org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer", parameters); + + // StorageDescriptor has an empty list of fields - SerDe will report them. + StorageDescriptor sd = + new StorageDescriptor( + Lists.newArrayList(), + location, + "org.apache.hadoop.mapred.TextInputFormat", + "org.apache.hadoop.mapred.TextOutputFormat", + false, + -1, + serDeInfo, + Lists.newArrayList(), + Lists.newArrayList(), + Maps.newHashMap()); + + Table hiveTable = + new Table( + hiveViewName, + dbName, + "test_owner", + 0, + 0, + 0, + sd, + Lists.newArrayList(), + Maps.newHashMap(), + "viewOriginalText", + "viewExpandedText", + TableType.VIRTUAL_VIEW.name()); + return hiveTable; + } +} diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveViewCommits.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveViewCommits.java new file mode 100644 index 000000000000..47abb51602fa --- /dev/null +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveViewCommits.java @@ -0,0 +1,516 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.hive; + +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyBoolean; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; + +import java.io.File; +import java.io.IOException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.Schema; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.CommitStateUnknownException; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.view.BaseView; +import org.apache.iceberg.view.View; +import org.apache.iceberg.view.ViewMetadata; +import org.apache.thrift.TException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +/** Test Hive locks and Hive errors and retry during commits. */ +public class TestHiveViewCommits { + + private static final String VIEW_NAME = "test_iceberg_view"; + private static final String DB_NAME = "hivedb"; + private static final Namespace NS = Namespace.of(DB_NAME); + private static final Schema SCHEMA = + new Schema( + 5, + required(3, "id", Types.IntegerType.get(), "unique ID"), + required(4, "data", Types.StringType.get())); + private static final TableIdentifier VIEW_IDENTIFIER = TableIdentifier.of(NS, VIEW_NAME); + + @RegisterExtension + protected static final HiveMetastoreExtension HIVE_METASTORE_EXTENSION = + HiveMetastoreExtension.builder().withDatabase(DB_NAME).build(); + + private View view; + private Path viewLocation; + + private static HiveCatalog catalog; + + @BeforeAll + public static void initCatalog() { + catalog = + (HiveCatalog) + CatalogUtil.loadCatalog( + HiveCatalog.class.getName(), + CatalogUtil.ICEBERG_CATALOG_TYPE_HIVE, + ImmutableMap.of( + CatalogProperties.CLIENT_POOL_CACHE_EVICTION_INTERVAL_MS, + String.valueOf(TimeUnit.SECONDS.toMillis(10))), + HIVE_METASTORE_EXTENSION.hiveConf()); + } + + @BeforeEach + public void createTestView() { + view = + catalog + .buildView(VIEW_IDENTIFIER) + .withSchema(SCHEMA) + .withDefaultNamespace(NS) + .withQuery("hive", "select * from ns.tbl") + .create(); + viewLocation = new Path(view.location()); + } + + @AfterEach + public void dropTestView() throws IOException { + viewLocation.getFileSystem(HIVE_METASTORE_EXTENSION.hiveConf()).delete(viewLocation, true); + catalog.dropView(VIEW_IDENTIFIER); + } + + @Test + public void testSuppressUnlockExceptions() { + HiveViewOperations ops = (HiveViewOperations) ((BaseView) view).operations(); + ViewMetadata metadataV1 = ops.current(); + assertThat(metadataV1.properties()).hasSize(0); + + view.updateProperties().set("k1", "v1").commit(); + ops.refresh(); + ViewMetadata metadataV2 = ops.current(); + assertThat(metadataV2.properties()).hasSize(1).containsEntry("k1", "v1"); + + HiveViewOperations spyOps = spy(ops); + + AtomicReference lockRef = new AtomicReference<>(); + + when(spyOps.lockObject()) + .thenAnswer( + i -> { + HiveLock lock = (HiveLock) i.callRealMethod(); + lockRef.set(lock); + return lock; + }); + + try { + spyOps.commit(metadataV2, metadataV1); + HiveLock spyLock = spy(lockRef.get()); + doThrow(new RuntimeException()).when(spyLock).unlock(); + } finally { + lockRef.get().unlock(); + } + + ops.refresh(); + + // the commit must succeed + assertThat(ops.current().properties()).hasSize(0).isEqualTo(metadataV1.properties()); + } + + /** + * Pretends we throw an error while persisting, and not found with check state, commit state + * should be treated as unknown, because in reality the persisting may still succeed, just not yet + * by the time of checking. + */ + @Test + public void testThriftExceptionUnknownStateIfNotInHistoryFailureOnCommit() + throws TException, InterruptedException { + HiveViewOperations ops = (HiveViewOperations) ((BaseView) view).operations(); + ViewMetadata metadataV1 = ops.current(); + assertThat(metadataV1.properties()).hasSize(0); + + view.updateProperties().set("k1", "v1").commit(); + ops.refresh(); + ViewMetadata metadataV2 = ops.current(); + assertThat(metadataV2.properties()).hasSize(1).containsEntry("k1", "v1"); + + HiveViewOperations spyOps = spy(ops); + + failCommitAndThrowException(spyOps); + + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + .isInstanceOf(CommitStateUnknownException.class) + .hasMessageStartingWith("Datacenter on fire"); + + ops.refresh(); + + assertThat(ops.current()).as("Current metadata should not have changed").isEqualTo(metadataV2); + assertThat(metadataFileExists(metadataV2)).as("Current metadata should still exist").isTrue(); + assertThat(metadataFileCount(metadataV2)) + .as( + "New metadata files should still exist, new location not in history but" + + " the commit may still succeed") + .isEqualTo(2); + } + + /** Pretends we throw an error while persisting that actually does commit serverside. */ + @Test + public void testThriftExceptionSuccessOnCommit() throws TException, InterruptedException { + HiveViewOperations ops = (HiveViewOperations) ((BaseView) view).operations(); + + ViewMetadata metadataV1 = ops.current(); + assertThat(metadataV1.properties()).hasSize(0); + + view.updateProperties().set("k1", "v1").commit(); + ops.refresh(); + ViewMetadata metadataV2 = ops.current(); + assertThat(metadataV2.properties()).hasSize(1).containsEntry("k1", "v1"); + + HiveViewOperations spyOps = spy(ops); + + // Simulate a communication error after a successful commit + commitAndThrowException(ops, spyOps); + spyOps.commit(metadataV2, metadataV1); + + assertThat(ops.current()).as("Current metadata should have not changed").isEqualTo(metadataV2); + assertThat(metadataFileExists(metadataV2)) + .as("Current metadata file should still exist") + .isTrue(); + assertThat(metadataFileCount(metadataV2)) + .as("Commit should have been successful and new metadata file should be made") + .isEqualTo(2); + } + + /** + * Pretends we throw an exception while persisting and don't know what happened, can't check to + * find out, but in reality the commit failed + */ + @Test + public void testThriftExceptionUnknownFailedCommit() throws TException, InterruptedException { + HiveViewOperations ops = (HiveViewOperations) ((BaseView) view).operations(); + ViewMetadata metadataV1 = ops.current(); + assertThat(metadataV1.properties()).hasSize(0); + + view.updateProperties().set("k1", "v1").commit(); + ops.refresh(); + ViewMetadata metadataV2 = ops.current(); + assertThat(metadataV2.properties()).hasSize(1).containsEntry("k1", "v1"); + + HiveViewOperations spyOps = spy(ops); + + failCommitAndThrowException(spyOps); + breakFallbackCatalogCommitCheck(spyOps); + + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + .isInstanceOf(CommitStateUnknownException.class) + .hasMessageStartingWith("Datacenter on fire"); + + ops.refresh(); + + assertThat(ops.current()).as("Current metadata should not have changed").isEqualTo(metadataV2); + assertThat(metadataFileExists(metadataV2)) + .as("Current metadata file should still exist") + .isTrue(); + assertThat(metadataFileCount(metadataV2)) + .as("Client could not determine outcome so new metadata file should also exist") + .isEqualTo(2); + } + + /** + * Pretends we throw an exception while persisting and don't know what happened, can't check to + * find out, but in reality the commit succeeded + */ + @Test + public void testThriftExceptionsUnknownSuccessCommit() throws TException, InterruptedException { + HiveViewOperations ops = (HiveViewOperations) ((BaseView) view).operations(); + ViewMetadata metadataV1 = ops.current(); + assertThat(metadataV1.properties()).hasSize(0); + + view.updateProperties().set("k1", "v1").commit(); + ops.refresh(); + + ViewMetadata metadataV2 = ops.current(); + + assertThat(metadataV2.properties()).hasSize(1); + + HiveViewOperations spyOps = spy(ops); + + commitAndThrowException(ops, spyOps); + breakFallbackCatalogCommitCheck(spyOps); + + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + .isInstanceOf(CommitStateUnknownException.class) + .hasMessageStartingWith("Datacenter on fire"); + + ops.refresh(); + ViewMetadata metadataV3 = ops.current(); + + assertThat(metadataV3).as("Current metadata should have changed").isNotEqualTo(metadataV2); + assertThat(metadataFileExists(metadataV3)) + .as("Current metadata file should still exist") + .isTrue(); + assertThat(metadataFileCount(metadataV3)) + .as("Commit should have been successful with updated properties at metadataV2") + .isEqualTo(2); + } + + /** + * Pretends we threw an exception while persisting, the commit succeeded, the lock expired, and a + * second committer placed a commit on top of ours before the first committer was able to check if + * their commit succeeded or not + * + *

Timeline: + * + *

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

This tests to make sure a disconnected client 1 doesn't think their commit failed just + * because it isn't the current one during the recheck phase. + */ + @Test + public void testThriftExceptionConcurrentCommit() throws TException, InterruptedException { + HiveViewOperations ops = (HiveViewOperations) ((BaseView) view).operations(); + ViewMetadata metadataV1 = ops.current(); + assertThat(metadataV1.properties()).hasSize(0); + + view.updateProperties().set("k0", "v0").commit(); + ops.refresh(); + ViewMetadata metadataV2 = ops.current(); + assertThat(metadataV2.properties()).hasSize(1).containsEntry("k0", "v0"); + + HiveViewOperations spyOps = spy(ops); + + AtomicReference lock = new AtomicReference<>(); + doAnswer( + l -> { + lock.set(ops.lockObject()); + return lock.get(); + }) + .when(spyOps) + .lockObject(); + + concurrentCommitAndThrowException(ops, spyOps, (BaseView) view, lock); + + // This commit should fail and concurrent commit should succeed even though this commit + // throws an exception after the persist operation succeeds + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + .isInstanceOf(CommitStateUnknownException.class) + .hasMessageContaining("Datacenter on fire"); + + ops.refresh(); + ViewMetadata metadataV3 = ops.current(); + + assertThat(metadataV3).as("Current metadata should have changed").isNotEqualTo(metadataV2); + assertThat(metadataFileExists(metadataV3)) + .as("Current metadata file should still exist") + .isTrue(); + assertThat(metadataV3.properties()) + .as("The new properties from the concurrent commit should have been successful") + .hasSize(2); + } + + @Test + public void testInvalidObjectException() { + TableIdentifier badTi = TableIdentifier.of(DB_NAME, "`test_iceberg_view`"); + assertThatThrownBy( + () -> + catalog + .buildView(badTi) + .withSchema(SCHEMA) + .withDefaultNamespace(NS) + .withQuery("hive", "select * from ns.tbl") + .create()) + .isInstanceOf(ValidationException.class) + .hasMessage("Invalid Hive object for " + DB_NAME + "." + "`test_iceberg_view`"); + } + + /** Uses NoLock and pretends we throw an error because of a concurrent commit */ + @Test + public void testNoLockThriftExceptionConcurrentCommit() throws TException, InterruptedException { + HiveViewOperations ops = (HiveViewOperations) ((BaseView) view).operations(); + ViewMetadata metadataV1 = ops.current(); + assertThat(metadataV1.properties()).hasSize(0); + + view.updateProperties().set("k1", "v1").commit(); + ops.refresh(); + ViewMetadata metadataV2 = ops.current(); + assertThat(metadataV2.properties()).hasSize(1).containsEntry("k1", "v1"); + + HiveViewOperations spyOps = spy(ops); + + // Sets NoLock + doReturn(new NoLock()).when(spyOps).lockObject(); + + // Simulate a concurrent view modification error + doThrow( + new RuntimeException( + "MetaException(message:The table has been modified. The parameter value for key 'metadata_location' is")) + .when(spyOps) + .persistTable(any(), anyBoolean(), any()); + + // Should throw a CommitFailedException so the commit could be retried + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + .isInstanceOf(CommitFailedException.class) + .hasMessageContaining("The view hivedb.test_iceberg_view has been modified concurrently"); + + ops.refresh(); + + assertThat(ops.current()).as("Current metadata should not have changed").isEqualTo(metadataV2); + assertThat(metadataFileExists(metadataV2)).as("Current metadata should still exist").isTrue(); + assertThat(metadataFileCount(metadataV2)) + .as("New metadata files should not exist") + .isEqualTo(1); + } + + @Test + public void testLockExceptionUnknownSuccessCommit() throws TException, InterruptedException { + HiveViewOperations ops = (HiveViewOperations) ((BaseView) view).operations(); + ViewMetadata metadataV1 = ops.current(); + assertThat(metadataV1.properties()).hasSize(0); + + view.updateProperties().set("k1", "v1").commit(); + ops.refresh(); + ViewMetadata metadataV2 = ops.current(); + assertThat(metadataV2.properties()).hasSize(1).containsEntry("k1", "v1"); + + HiveViewOperations spyOps = spy(ops); + + // Simulate a communication error after a successful commit + doAnswer( + i -> { + org.apache.hadoop.hive.metastore.api.Table tbl = + i.getArgument(0, org.apache.hadoop.hive.metastore.api.Table.class); + String location = i.getArgument(2, String.class); + ops.persistTable(tbl, true, location); + throw new LockException("Datacenter on fire"); + }) + .when(spyOps) + .persistTable(any(), anyBoolean(), any()); + + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + .hasMessageContaining("Failed to heartbeat for hive lock while") + .isInstanceOf(CommitStateUnknownException.class); + + ops.refresh(); + + assertThat(metadataV2.location()) + .as("Current metadata should have changed to metadata V1") + .isEqualTo(metadataV1.location()); + assertThat(metadataFileExists(metadataV2)) + .as("Current metadata file should still exist") + .isTrue(); + assertThat(metadataFileCount(metadataV2)).as("New metadata file should exist").isEqualTo(2); + } + + @Test + public void testCommitExceptionWithoutMessage() throws TException, InterruptedException { + HiveViewOperations ops = (HiveViewOperations) ((BaseView) view).operations(); + ViewMetadata metadataV1 = ops.current(); + assertThat(metadataV1.properties()).hasSize(0); + + view.updateProperties().set("k1", "v1").commit(); + + ops.refresh(); + + HiveViewOperations spyOps = spy(ops); + doThrow(new RuntimeException()).when(spyOps).persistTable(any(), anyBoolean(), any()); + + assertThatThrownBy(() -> spyOps.commit(ops.current(), metadataV1)) + .isInstanceOf(CommitStateUnknownException.class) + .hasMessageStartingWith("null\nCannot determine whether the commit was successful or not"); + } + + private void commitAndThrowException( + HiveViewOperations realOperations, HiveViewOperations spyOperations) + throws TException, InterruptedException { + // Simulate a communication error after a successful commit + doAnswer( + i -> { + org.apache.hadoop.hive.metastore.api.Table tbl = + i.getArgument(0, org.apache.hadoop.hive.metastore.api.Table.class); + String location = i.getArgument(2, String.class); + realOperations.persistTable(tbl, true, location); + throw new TException("Datacenter on fire"); + }) + .when(spyOperations) + .persistTable(any(), anyBoolean(), any()); + } + + private void concurrentCommitAndThrowException( + HiveViewOperations realOperations, + HiveViewOperations spyOperations, + BaseView baseView, + AtomicReference lock) + throws TException, InterruptedException { + // Simulate a communication error after a successful commit + doAnswer( + i -> { + org.apache.hadoop.hive.metastore.api.Table tbl = + i.getArgument(0, org.apache.hadoop.hive.metastore.api.Table.class); + String location = i.getArgument(2, String.class); + realOperations.persistTable(tbl, true, location); + // Simulate lock expiration or removal + lock.get().unlock(); + baseView.operations().refresh(); + baseView.updateProperties().set("k1", "v1").set("k2", "v2").commit(); + throw new TException("Datacenter on fire"); + }) + .when(spyOperations) + .persistTable(any(), anyBoolean(), any()); + } + + private void failCommitAndThrowException(HiveViewOperations spyOperations) + throws TException, InterruptedException { + doThrow(new TException("Datacenter on fire")) + .when(spyOperations) + .persistTable(any(), anyBoolean(), any()); + } + + private void breakFallbackCatalogCommitCheck(HiveViewOperations spyOperations) { + when(spyOperations.refresh()) + .thenThrow(new RuntimeException("Still on fire")); // Failure on commit check + } + + private boolean metadataFileExists(ViewMetadata metadata) { + return new File(metadata.metadataFileLocation().replace("file:", "")).exists(); + } + + private int metadataFileCount(ViewMetadata metadata) { + return new File(metadata.metadataFileLocation().replace("file:", "")) + .getParentFile() + .listFiles(file -> file.getName().endsWith("metadata.json")) + .length; + } +} diff --git a/hive-runtime/build.gradle b/hive-runtime/build.gradle index 6f891be8087e..a107afcb3777 100644 --- a/hive-runtime/build.gradle +++ b/hive-runtime/build.gradle @@ -20,7 +20,7 @@ def hiveVersions = (System.getProperty("hiveVersions") != null ? System.getProperty("hiveVersions") : System.getProperty("defaultHiveVersions")).split(",") project(':iceberg-hive-runtime') { - apply plugin: 'io.github.goooler.shadow' + apply plugin: 'com.gradleup.shadow' tasks.jar.dependsOn tasks.shadowJar diff --git a/hive3-orc-bundle/build.gradle b/hive3-orc-bundle/build.gradle index 3ca89bd47435..4e1e9c5dd222 100644 --- a/hive3-orc-bundle/build.gradle +++ b/hive3-orc-bundle/build.gradle @@ -21,7 +21,7 @@ // name. This is to be used by Hive3 for features including e.g. vectorization. project(':iceberg-hive3-orc-bundle') { - apply plugin: 'io.github.goooler.shadow' + apply plugin: 'com.gradleup.shadow' tasks.jar.dependsOn tasks.shadowJar diff --git a/hive3/src/main/java/org/apache/hadoop/hive/ql/io/orc/OrcSplit.java b/hive3/src/main/java/org/apache/hadoop/hive/ql/io/orc/OrcSplit.java index 4f2cdf0282e2..4031bfaa20f6 100644 --- a/hive3/src/main/java/org/apache/hadoop/hive/ql/io/orc/OrcSplit.java +++ b/hive3/src/main/java/org/apache/hadoop/hive/ql/io/orc/OrcSplit.java @@ -51,6 +51,7 @@ public class OrcSplit extends FileSplit implements ColumnarSplit, LlapAwareSplit private static final Logger LOG = LoggerFactory.getLogger(OrcSplit.class); private OrcTail orcTail; private boolean hasFooter; + /** This means {@link AcidUtils.AcidBaseFileType#ORIGINAL_BASE} */ private boolean isOriginal; diff --git a/kafka-connect/build.gradle b/kafka-connect/build.gradle index 60aaeee8c6f1..d38d01768e66 100644 --- a/kafka-connect/build.gradle +++ b/kafka-connect/build.gradle @@ -70,9 +70,9 @@ project(':iceberg-kafka-connect:iceberg-kafka-connect-runtime') { // force upgrades for dependencies with known vulnerabilities... resolutionStrategy { force 'org.codehaus.jettison:jettison:1.5.4' - force 'org.xerial.snappy:snappy-java:1.1.10.6' - force 'org.apache.commons:commons-compress:1.27.0' - force 'org.apache.hadoop.thirdparty:hadoop-shaded-guava:1.2.0' + force 'org.xerial.snappy:snappy-java:1.1.10.7' + force 'org.apache.commons:commons-compress:1.27.1' + force 'org.apache.hadoop.thirdparty:hadoop-shaded-guava:1.3.0' } } } @@ -115,6 +115,7 @@ project(':iceberg-kafka-connect:iceberg-kafka-connect-runtime') { implementation platform(libs.awssdk.bom) implementation 'software.amazon.awssdk:apache-client' implementation 'software.amazon.awssdk:auth' + implementation "software.amazon.awssdk:http-auth-aws-crt" implementation 'software.amazon.awssdk:iam' implementation 'software.amazon.awssdk:sso' implementation 'software.amazon.awssdk:s3' @@ -197,7 +198,8 @@ project(':iceberg-kafka-connect:iceberg-kafka-connect-runtime') { from configurations.runtimeClasspath } into('doc/') { - from "$rootDir/LICENSE" + from "$projectDir/LICENSE" + from "$projectDir/NOTICE" } into('assets/') { from "${processResources.destinationDir}/iceberg.png" @@ -211,7 +213,8 @@ project(':iceberg-kafka-connect:iceberg-kafka-connect-runtime') { from configurations.hive } into('doc/') { - from "$rootDir/LICENSE" + from "$projectDir/LICENSE" + from "$projectDir/NOTICE" } into('assets/') { from "${processResources.destinationDir}/iceberg.png" @@ -220,6 +223,11 @@ project(':iceberg-kafka-connect:iceberg-kafka-connect-runtime') { } } + // there are no Maven artifacts so disable publishing tasks... + project.afterEvaluate { + project.tasks.matching { it.group == 'publishing' }.each {it.enabled = false} + } + tasks.jar.enabled = false tasks.distTar.enabled = false diff --git a/kafka-connect/kafka-connect-runtime/LICENSE b/kafka-connect/kafka-connect-runtime/LICENSE new file mode 100644 index 000000000000..6eb4fdee65b8 --- /dev/null +++ b/kafka-connect/kafka-connect-runtime/LICENSE @@ -0,0 +1,1970 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + +-------------------------------------------------------------------------------- + +This binary artifact contains code from the following projects: + +-------------------------------------------------------------------------------- + +Group: aopalliance Name: aopalliance Version: 1.0 +Project URL (from POM): http://aopalliance.sourceforge.net +License (from POM): Public Domain +-------------------------------------------------------------------------------- + +Group: ch.qos.reload4j Name: reload4j Version: 1.2.22 +Project URL (from manifest): https://reload4j.qos.ch/ +Project URL (from POM): https://reload4j.qos.ch +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.azure Name: azure-core Version: 1.49.1 +Project URL (from POM): https://github.com/Azure/azure-sdk-for-java +License (from POM): The MIT License (MIT) - http://opensource.org/licenses/MIT + +-------------------------------------------------------------------------------- + +Group: com.azure Name: azure-core-http-netty Version: 1.15.1 +Project URL (from POM): https://github.com/Azure/azure-sdk-for-java +License (from POM): The MIT License (MIT) - http://opensource.org/licenses/MIT + +-------------------------------------------------------------------------------- + +Group: com.azure Name: azure-identity Version: 1.13.0 +Project URL (from POM): https://github.com/Azure/azure-sdk-for-java +License (from POM): The MIT License (MIT) - http://opensource.org/licenses/MIT + +-------------------------------------------------------------------------------- + +Group: com.azure Name: azure-json Version: 1.1.0 +Project URL (from POM): https://github.com/Azure/azure-sdk-for-java +License (from POM): The MIT License (MIT) - http://opensource.org/licenses/MIT + +-------------------------------------------------------------------------------- + +Group: com.azure Name: azure-storage-blob Version: 12.26.1 +Project URL (from POM): https://github.com/Azure/azure-sdk-for-java +License (from POM): The MIT License (MIT) - http://opensource.org/licenses/MIT + +-------------------------------------------------------------------------------- + +Group: com.azure Name: azure-storage-common Version: 12.25.1 +Project URL (from POM): https://github.com/Azure/azure-sdk-for-java +License (from POM): The MIT License (MIT) - http://opensource.org/licenses/MIT + +-------------------------------------------------------------------------------- + +Group: com.azure Name: azure-storage-file-datalake Version: 12.19.1 +Project URL (from POM): https://github.com/Azure/azure-sdk-for-java +License (from POM): The MIT License (MIT) - http://opensource.org/licenses/MIT + +-------------------------------------------------------------------------------- + +Group: com.azure Name: azure-storage-internal-avro Version: 12.11.1 +Project URL (from POM): https://github.com/Azure/azure-sdk-for-java +License (from POM): The MIT License (MIT) - http://opensource.org/licenses/MIT + +-------------------------------------------------------------------------------- + +Group: com.azure Name: azure-xml Version: 1.0.0 +Project URL (from POM): https://github.com/Azure/azure-sdk-for-java +License (from POM): The MIT License (MIT) - http://opensource.org/licenses/MIT + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.jackson.core Name: jackson-annotations Version: 2.17.2 +Project URL (from manifest): https://github.com/FasterXML/jackson +Project URL (from POM): https://github.com/FasterXML/jackson +License (from POM): The Apache Software License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.jackson.core Name: jackson-core Version: 2.17.2 +Project URL (from manifest): https://github.com/FasterXML/jackson-core +Project URL (from POM): https://github.com/FasterXML/jackson-core +License (from POM): Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.jackson.core Name: jackson-databind Version: 2.17.2 +Project URL (from manifest): https://github.com/FasterXML/jackson +Project URL (from POM): https://github.com/FasterXML/jackson +License (from POM): Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.jackson.dataformat Name: jackson-dataformat-xml Version: 2.17.2 +Project URL (from manifest): https://github.com/FasterXML/jackson-dataformat-xml +Project URL (from POM): https://github.com/FasterXML/jackson-dataformat-xml +License (from POM): Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.jackson.datatype Name: jackson-datatype-jsr310 Version: 2.17.2 +Project URL (from manifest): https://github.com/FasterXML/jackson-modules-java8/jackson-datatype-jsr310 +License (from POM): Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.jackson.jaxrs Name: jackson-jaxrs-base Version: 2.17.2 +Project URL (from manifest): https://github.com/FasterXML/jackson-jaxrs-providers/jackson-jaxrs-base +License (from POM): Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.jackson.jaxrs Name: jackson-jaxrs-json-provider Version: 2.17.2 +Project URL (from manifest): https://github.com/FasterXML/jackson-jaxrs-providers/jackson-jaxrs-json-provider +License (from POM): Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.jackson.module Name: jackson-module-jaxb-annotations Version: 2.17.2 +Project URL (from manifest): https://github.com/FasterXML/jackson-modules-base +Project URL (from POM): https://github.com/FasterXML/jackson-modules-base +License (from POM): Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.woodstox Name: woodstox-core Version: 6.7.0 +Project URL (from manifest): https://github.com/FasterXML/woodstox +Project URL (from POM): https://github.com/FasterXML/woodstox +License (from POM): The Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.github.ben-manes.caffeine Name: caffeine Version: 2.9.3 +Project URL (from POM): https://github.com/ben-manes/caffeine +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.github.luben Name: zstd-jni Version: 1.5.0-1 +License URL (from manifest): https://opensource.org/licenses/BSD-2-Clause +Project URL (from POM): https://github.com/luben/zstd-jni +License (from POM): BSD 2-Clause License - https://opensource.org/licenses/BSD-2-Clause + +-------------------------------------------------------------------------------- + +Group: com.github.pjfanning Name: jersey-json Version: 1.20 +Project URL (from POM): https://github.com/pjfanning/jersey-1.x +License (from POM): CDDL 1.1 - http://glassfish.java.net/public/CDDL+GPL_1_1.html +License (from POM): GPL2 w/ CPE - http://glassfish.java.net/public/CDDL+GPL_1_1.html + +-------------------------------------------------------------------------------- + +Group: com.github.stephenc.jcip Name: jcip-annotations Version: 1.0-1 +Project URL (from POM): http://stephenc.github.com/jcip-annotations +License (from POM): Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.android Name: annotations Version: 4.1.1.4 +Project URL (from POM): http://source.android.com/ +License (from POM): Apache 2.0 - http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: com.google.api Name: api-common Version: 2.33.0 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): BSD-3-Clause - https://github.com/googleapis/api-common-java/blob/main/LICENSE + +-------------------------------------------------------------------------------- + +Group: com.google.api Name: gax Version: 2.50.0 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): BSD-3-Clause - https://github.com/googleapis/gax-java/blob/master/LICENSE + +-------------------------------------------------------------------------------- + +Group: com.google.api Name: gax-grpc Version: 2.50.0 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): BSD-3-Clause - https://github.com/googleapis/gax-java/blob/master/LICENSE + +-------------------------------------------------------------------------------- + +Group: com.google.api Name: gax-httpjson Version: 2.50.0 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): BSD-3-Clause - https://github.com/googleapis/gax-java/blob/master/LICENSE + +-------------------------------------------------------------------------------- + +Group: com.google.api-client Name: google-api-client Version: 2.6.0 +Project URL (from manifest): https://developers.google.com/api-client-library/java/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.api.grpc Name: gapic-google-cloud-storage-v2 Version: 2.40.1-alpha +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.api.grpc Name: grpc-google-cloud-storage-v2 Version: 2.40.1-alpha +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.api.grpc Name: proto-google-cloud-storage-v2 Version: 2.40.1-alpha +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.api.grpc Name: proto-google-common-protos Version: 2.41.0 +Project URL (from POM): https://github.com/googleapis/sdk-platform-java +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.api.grpc Name: proto-google-iam-v1 Version: 1.36.0 +Project URL (from POM): https://github.com/googleapis/sdk-platform-java +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.apis Name: google-api-services-storage Version: v1-rev20240621-2.0.0 +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.auth Name: google-auth-library-credentials Version: 1.23.0 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): BSD New license - http://opensource.org/licenses/BSD-3-Clause + +-------------------------------------------------------------------------------- + +Group: com.google.auth Name: google-auth-library-oauth2-http Version: 1.23.0 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): BSD New license - http://opensource.org/licenses/BSD-3-Clause + +-------------------------------------------------------------------------------- + +Group: com.google.auto.value Name: auto-value-annotations Version: 1.10.4 +Project URL (from POM): https://github.com/google/auto/tree/main/value +License (from POM): Apache 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.cloud Name: google-cloud-core Version: 2.40.0 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.cloud Name: google-cloud-core-grpc Version: 2.40.0 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.cloud Name: google-cloud-core-http Version: 2.40.0 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.cloud Name: google-cloud-storage Version: 2.40.1 +Project URL (from POM): https://github.com/googleapis/java-storage +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.code.findbugs Name: jsr305 Version: 3.0.2 +Project URL (from POM): http://findbugs.sourceforge.net/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.code.gson Name: gson Version: 2.11.0 +Project URL (from manifest): https://github.com/google/gson +Manifest License: "Apache-2.0";link="https://www.apache.org/licenses/LICENSE-2.0.txt" (Not packaged) +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.errorprone Name: error_prone_annotations Version: 2.28.0 +Project URL (from manifest): https://errorprone.info/error_prone_annotations +Manifest License: "Apache 2.0";link="http://www.apache.org/licenses/LICENSE-2.0.txt" (Not packaged) +License (from POM): Apache 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.guava Name: failureaccess Version: 1.0.2 +Project URL (from manifest): https://github.com/google/guava/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.guava Name: guava Version: 33.1.0-jre +Project URL (from manifest): https://github.com/google/guava/ +Project URL (from POM): https://github.com/google/guava +License (from POM): Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.guava Name: listenablefuture Version: 9999.0-empty-to-avoid-conflict-with-guava +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.http-client Name: google-http-client Version: 1.44.2 +Project URL (from manifest): https://www.google.com/ +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.http-client Name: google-http-client-apache-v2 Version: 1.44.2 +Project URL (from manifest): https://www.google.com/ +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.http-client Name: google-http-client-appengine Version: 1.44.2 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.http-client Name: google-http-client-gson Version: 1.44.2 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.http-client Name: google-http-client-jackson2 Version: 1.44.2 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.inject Name: guice Version: 4.0 +Project URL (from manifest): https://github.com/google/guice +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.j2objc Name: j2objc-annotations Version: 3.0.0 +Project URL (from POM): https://github.com/google/j2objc/ +License (from POM): Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.oauth-client Name: google-oauth-client Version: 1.36.0 +Project URL (from manifest): https://www.google.com/ +License (from POM): The Apache Software License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.protobuf Name: protobuf-java Version: 3.25.3 +Project URL (from manifest): https://developers.google.com/protocol-buffers/ +License (from POM): BSD-3-Clause - https://opensource.org/licenses/BSD-3-Clause + +-------------------------------------------------------------------------------- + +Group: com.google.protobuf Name: protobuf-java-util Version: 3.25.3 +Project URL (from manifest): https://developers.google.com/protocol-buffers/ +License (from POM): BSD-3-Clause - https://opensource.org/licenses/BSD-3-Clause + +-------------------------------------------------------------------------------- + +Group: com.google.re2j Name: re2j Version: 1.7 +Project URL (from POM): https://github.com/google/re2j +License (from POM): Go License - https://golang.org/LICENSE + +-------------------------------------------------------------------------------- + +Group: com.jcraft Name: jsch Version: 0.1.55 +Project URL (from POM): http://www.jcraft.com/jsch/ +License (from POM): Revised BSD - http://www.jcraft.com/jsch/LICENSE.txt + +-------------------------------------------------------------------------------- + +Group: com.jolbox Name: bonecp Version: 0.8.0.RELEASE +Project URL (from manifest): http://jolbox.com +License (from POM): Apache v2 - http://www.apache.org/licenses/LICENSE-2.0.html + +-------------------------------------------------------------------------------- + +Group: com.microsoft.azure Name: msal4j Version: 1.15.1 +Project URL (from manifest): https://github.com/AzureAD/microsoft-authentication-library-for-java +Manifest License: "MIT License" (Not packaged) +Project URL (from POM): https://github.com/AzureAD/microsoft-authentication-library-for-java +License (from POM): MIT License +-------------------------------------------------------------------------------- + +Group: com.microsoft.azure Name: msal4j-persistence-extension Version: 1.3.0 +Project URL (from POM): https://github.com/AzureAD/microsoft-authentication-library-for-java +License (from POM): MIT License +-------------------------------------------------------------------------------- + +Group: com.nimbusds Name: content-type Version: 2.3 +Project URL (from manifest): https://connect2id.com +Project URL (from POM): https://bitbucket.org/connect2id/nimbus-content-type +License (from POM): The Apache Software License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.nimbusds Name: lang-tag Version: 1.7 +Project URL (from manifest): https://connect2id.com/ +Project URL (from POM): https://bitbucket.org/connect2id/nimbus-language-tags +License (from POM): The Apache Software License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.nimbusds Name: nimbus-jose-jwt Version: 9.37.3 +Project URL (from manifest): https://connect2id.com +Project URL (from POM): https://bitbucket.org/connect2id/nimbus-jose-jwt +License (from POM): The Apache Software License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.nimbusds Name: oauth2-oidc-sdk Version: 11.9.1 +Project URL (from manifest): https://bitbucket.org/connect2id/oauth-2.0-sdk-with-openid-connect-extensions +Manifest License: "Apache License, version 2.0";link="https://www.apache.org/licenses/LICENSE-2.0.html" (Not packaged) +Project URL (from POM): https://bitbucket.org/connect2id/oauth-2.0-sdk-with-openid-connect-extensions +License (from POM): Apache License, version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.html + +-------------------------------------------------------------------------------- + +Group: com.squareup.okhttp3 Name: okhttp Version: 4.9.3 +Project URL (from POM): https://square.github.io/okhttp/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.squareup.okio Name: okio Version: 2.8.0 +Project URL (from POM): https://github.com/square/okio/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.sun.xml.bind Name: jaxb-impl Version: 2.2.3-1 +Project URL (from POM): http://jaxb.java.net/ +License (from POM): CDDL 1.1 - https://glassfish.java.net/public/CDDL+GPL_1_1.html +License (from POM): GPL2 w/ CPE - https://glassfish.java.net/public/CDDL+GPL_1_1.html + +-------------------------------------------------------------------------------- + +Group: commons-beanutils Name: commons-beanutils Version: 1.9.4 +Project URL (from manifest): https://commons.apache.org/proper/commons-beanutils/ +Project URL (from POM): https://commons.apache.org/proper/commons-beanutils/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: commons-cli Name: commons-cli Version: 1.2 +Project URL (from manifest): http://commons.apache.org/cli/ +Project URL (from POM): http://commons.apache.org/cli/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: commons-codec Name: commons-codec Version: 1.17.1 +Project URL (from manifest): https://commons.apache.org/proper/commons-codec/ +Project URL (from POM): https://commons.apache.org/proper/commons-codec/ +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: commons-collections Name: commons-collections Version: 3.2.2 +Project URL (from manifest): http://commons.apache.org/collections/ +Project URL (from POM): http://commons.apache.org/collections/ +License (from POM): Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: commons-dbcp Name: commons-dbcp Version: 1.4 +Project URL (from manifest): http://commons.apache.org/dbcp/ +Project URL (from POM): http://commons.apache.org/dbcp/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: commons-io Name: commons-io Version: 2.16.1 +Project URL (from manifest): https://commons.apache.org/proper/commons-io/ +Project URL (from POM): https://commons.apache.org/proper/commons-io/ +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: commons-lang Name: commons-lang Version: 2.6 +Project URL (from manifest): http://commons.apache.org/lang/ +Project URL (from POM): http://commons.apache.org/lang/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: commons-logging Name: commons-logging Version: 1.2 +Project URL (from manifest): http://commons.apache.org/proper/commons-logging/ +Project URL (from POM): http://commons.apache.org/proper/commons-logging/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: commons-net Name: commons-net Version: 3.9.0 +Project URL (from manifest): https://commons.apache.org/proper/commons-net/ +Project URL (from POM): https://commons.apache.org/proper/commons-net/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: commons-pool Name: commons-pool Version: 1.6 +Project URL (from manifest): http://commons.apache.org/pool/ +Project URL (from POM): http://commons.apache.org/pool/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: dnsjava Name: dnsjava Version: 2.1.7 +Project URL (from POM): http://www.dnsjava.org +License (from POM): BSD 2-Clause license - http://opensource.org/licenses/BSD-2-Clause + +-------------------------------------------------------------------------------- + +Group: io.airlift Name: aircompressor Version: 0.27 +Project URL (from POM): https://github.com/airlift/aircompressor +License (from POM): Apache License 2.0 - https://www.apache.org/licenses/LICENSE-2.0.html + +-------------------------------------------------------------------------------- + +Group: io.dropwizard.metrics Name: metrics-core Version: 3.2.4 +License (from POM): Apache License 2.0 - http://www.apache.org/licenses/LICENSE-2.0.html + +-------------------------------------------------------------------------------- + +Group: io.dropwizard.metrics Name: metrics-json Version: 3.1.0 +License (from POM): Apache License 2.0 - http://www.apache.org/licenses/LICENSE-2.0.html + +-------------------------------------------------------------------------------- + +Group: io.dropwizard.metrics Name: metrics-jvm Version: 3.1.0 +License (from POM): Apache License 2.0 - http://www.apache.org/licenses/LICENSE-2.0.html + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-alts Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-api Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-auth Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-context Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-core Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-googleapis Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-grpclb Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-inprocess Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-netty-shaded Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-protobuf Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-protobuf-lite Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-rls Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-services Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-stub Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-util Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-xds Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-buffer Version: 4.1.111.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-codec Version: 4.1.111.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-codec-dns Version: 4.1.109.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-codec-http Version: 4.1.111.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-codec-http2 Version: 4.1.111.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-codec-socks Version: 4.1.110.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-common Version: 4.1.111.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-handler Version: 4.1.111.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-handler-proxy Version: 4.1.110.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-resolver Version: 4.1.111.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-resolver-dns Version: 4.1.109.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-resolver-dns-classes-macos Version: 4.1.109.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-resolver-dns-native-macos Version: 4.1.109.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-tcnative-boringssl-static Version: 2.0.65.Final +Project URL (from manifest): https://netty.io/ +Project URL (from POM): https://github.com/netty/netty-tcnative/netty-tcnative-boringssl-static/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-tcnative-classes Version: 2.0.65.Final +Project URL (from manifest): https://netty.io/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-transport Version: 4.1.111.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-transport-classes-epoll Version: 4.1.111.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-transport-classes-kqueue Version: 4.1.110.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-transport-native-epoll Version: 4.1.110.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-transport-native-kqueue Version: 4.1.110.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-transport-native-unix-common Version: 4.1.111.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.opencensus Name: opencensus-api Version: 0.31.1 +Project URL (from POM): https://github.com/census-instrumentation/opencensus-java +License (from POM): The Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: io.opencensus Name: opencensus-contrib-http-util Version: 0.31.1 +Project URL (from POM): https://github.com/census-instrumentation/opencensus-java +License (from POM): The Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: io.opencensus Name: opencensus-proto Version: 0.2.0 +Project URL (from POM): https://github.com/census-instrumentation/opencensus-proto +License (from POM): The Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: io.perfmark Name: perfmark-api Version: 0.27.0 +Project URL (from POM): https://github.com/perfmark/perfmark +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.projectreactor Name: reactor-core Version: 3.4.38 +Project URL (from POM): https://github.com/reactor/reactor-core +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: io.projectreactor.netty Name: reactor-netty-core Version: 1.0.45 +Project URL (from POM): https://github.com/reactor/reactor-netty +License (from POM): The Apache Software License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: io.projectreactor.netty Name: reactor-netty-http Version: 1.0.45 +Project URL (from POM): https://github.com/reactor/reactor-netty +License (from POM): The Apache Software License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: jakarta.activation Name: jakarta.activation-api Version: 1.2.1 +Project URL (from manifest): https://www.eclipse.org +License (from POM): EDL 1.0 - http://www.eclipse.org/org/documents/edl-v10.php +License (from POM): Eclipse Public License v. 2.0 - https://www.eclipse.org/org/documents/epl-2.0/EPL-2.0.txt +License (from POM): GNU General Public License, version 2 with the GNU Classpath Exception - https://www.gnu.org/software/classpath/license.html + +-------------------------------------------------------------------------------- + +Group: jakarta.activation Name: jakarta.activation-api Version: 1.2.2 +Project URL (from manifest): https://www.eclipse.org +License (from POM): EDL 1.0 - http://www.eclipse.org/org/documents/edl-v10.php +License (from POM): Eclipse Public License v. 2.0 - https://www.eclipse.org/org/documents/epl-2.0/EPL-2.0.txt +License (from POM): GNU General Public License, version 2 with the GNU Classpath Exception - https://www.gnu.org/software/classpath/license.html + +-------------------------------------------------------------------------------- + +Group: jakarta.xml.bind Name: jakarta.xml.bind-api Version: 2.3.3 +Project URL (from manifest): https://www.eclipse.org +License (from POM): Eclipse Distribution License - v 1.0 - http://www.eclipse.org/org/documents/edl-v10.php +License (from POM): Eclipse Public License v. 2.0 - https://www.eclipse.org/org/documents/epl-2.0/EPL-2.0.txt +License (from POM): GNU General Public License, version 2 with the GNU Classpath Exception - https://www.gnu.org/software/classpath/license.html + +-------------------------------------------------------------------------------- + +Group: javax.annotation Name: javax.annotation-api Version: 1.3.2 +Project URL (from manifest): https://javaee.github.io/glassfish +Project URL (from POM): http://jcp.org/en/jsr/detail?id=250 +License (from POM): CDDL + GPLv2 with classpath exception - https://github.com/javaee/javax.annotation/blob/master/LICENSE + +-------------------------------------------------------------------------------- + +Group: javax.inject Name: javax.inject Version: 1 +Project URL (from POM): http://code.google.com/p/atinject/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: javax.jdo Name: jdo-api Version: 3.0.1 +Project URL (from POM): http://db.apache.org/jdo +License (from POM): Apache 2 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: javax.servlet Name: javax.servlet-api Version: 3.1.0 +Project URL (from manifest): https://glassfish.dev.java.net +Project URL (from POM): http://servlet-spec.java.net +License (from POM): CDDL + GPLv2 with classpath exception - https://glassfish.dev.java.net/nonav/public/CDDL+GPL.html + +-------------------------------------------------------------------------------- + +Group: javax.servlet.jsp Name: jsp-api Version: 2.1 + +-------------------------------------------------------------------------------- + +Group: javax.transaction Name: jta Version: 1.1 +Project URL (from POM): http://java.sun.com/products/jta + +-------------------------------------------------------------------------------- + +Group: javax.xml.bind Name: jaxb-api Version: 2.2.11 +Project URL (from manifest): http://www.oracle.com/ +Project URL (from POM): http://jaxb.java.net/ +License (from POM): CDDL 1.1 - https://glassfish.java.net/public/CDDL+GPL_1_1.html +License (from POM): GPL2 w/ CPE - https://glassfish.java.net/public/CDDL+GPL_1_1.html + +-------------------------------------------------------------------------------- + +Group: javax.xml.bind Name: jaxb-api Version: 2.2.2 +Project URL (from POM): https://jaxb.dev.java.net/ +License (from POM): CDDL 1.1 - https://glassfish.dev.java.net/public/CDDL+GPL_1_1.html +License (from POM): GPL2 w/ CPE - https://glassfish.dev.java.net/public/CDDL+GPL_1_1.html + +-------------------------------------------------------------------------------- + +Group: javax.xml.stream Name: stax-api Version: 1.0-2 +License (from POM): COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0 - http://www.sun.com/cddl/cddl.html +License (from POM): GNU General Public Library - http://www.gnu.org/licenses/gpl.txt + +-------------------------------------------------------------------------------- + +Group: javolution Name: javolution Version: 5.5.1 +Project URL (from manifest): http://javolution.org +Project URL (from POM): http://javolution.org +License (from POM): BSD License - http://javolution.org/LICENSE.txt + +-------------------------------------------------------------------------------- + +Group: jline Name: jline Version: 2.12 +License (from POM): The BSD License - http://www.opensource.org/licenses/bsd-license.php + +-------------------------------------------------------------------------------- + +Group: joda-time Name: joda-time Version: 2.8.1 +Project URL (from manifest): http://www.joda.org/joda-time/ +Manifest License: Apache 2.0 (Not packaged) +Project URL (from POM): http://www.joda.org/joda-time/ +License (from POM): Apache 2 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: log4j Name: log4j Version: 1.2.17 +Project URL (from manifest): http://logging.apache.org/log4j/1.2 +Project URL (from POM): http://logging.apache.org/log4j/1.2/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: net.java.dev.jna Name: jna Version: 5.13.0 +Project URL (from POM): https://github.com/java-native-access/jna +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): LGPL-2.1-or-later - https://www.gnu.org/licenses/old-licenses/lgpl-2.1 + +-------------------------------------------------------------------------------- + +Group: net.java.dev.jna Name: jna-platform Version: 5.13.0 +Project URL (from POM): https://github.com/java-native-access/jna +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): LGPL-2.1-or-later - https://www.gnu.org/licenses/old-licenses/lgpl-2.1 + +-------------------------------------------------------------------------------- + +Group: net.minidev Name: accessors-smart Version: 2.5.0 +Project URL (from manifest): https://urielch.github.io/ +Project URL (from POM): https://urielch.github.io/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: net.minidev Name: json-smart Version: 2.5.0 +Project URL (from manifest): https://urielch.github.io/ +Project URL (from POM): https://urielch.github.io/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: net.sf.opencsv Name: opencsv Version: 2.3 +Project URL (from POM): http://opencsv.sf.net +License (from POM): Apache 2 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.antlr Name: antlr-runtime Version: 3.5.2 +Project URL (from POM): http://www.antlr.org +License (from POM): BSD licence - http://antlr.org/license.html + +-------------------------------------------------------------------------------- + +Group: org.apache.ant Name: ant Version: 1.9.1 +Project URL (from POM): http://ant.apache.org/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.ant Name: ant-launcher Version: 1.9.1 +Project URL (from POM): http://ant.apache.org/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.avro Name: avro Version: 1.12.0 +Project URL (from manifest): https://www.apache.org/ +Project URL (from POM): https://avro.apache.org +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.commons Name: commons-compress Version: 1.26.2 +Project URL (from manifest): https://commons.apache.org/proper/commons-compress/ +Project URL (from POM): https://commons.apache.org/proper/commons-compress/ +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.commons Name: commons-configuration2 Version: 2.8.0 +Project URL (from manifest): https://commons.apache.org/proper/commons-configuration/ +Project URL (from POM): https://commons.apache.org/proper/commons-configuration/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.commons Name: commons-lang3 Version: 3.14.0 +Project URL (from manifest): https://commons.apache.org/proper/commons-lang/ +Project URL (from POM): https://commons.apache.org/proper/commons-lang/ +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.commons Name: commons-math3 Version: 3.1.1 +Project URL (from manifest): http://commons.apache.org/math/ +Project URL (from POM): http://commons.apache.org/math/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.commons Name: commons-text Version: 1.10.0 +Project URL (from manifest): https://commons.apache.org/proper/commons-text +Project URL (from POM): https://commons.apache.org/proper/commons-text +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.curator Name: curator-client Version: 5.2.0 +Project URL (from manifest): http://www.apache.org/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.curator Name: curator-framework Version: 5.2.0 +Project URL (from manifest): http://www.apache.org/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.curator Name: curator-recipes Version: 5.2.0 +Project URL (from manifest): http://www.apache.org/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.derby Name: derby Version: 10.10.2.0 +License (from POM): Apache 2 - http://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hadoop Name: hadoop-annotations Version: 3.3.6 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hadoop Name: hadoop-auth Version: 3.3.6 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hadoop Name: hadoop-client Version: 3.3.6 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hadoop Name: hadoop-common Version: 3.3.6 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hadoop Name: hadoop-hdfs-client Version: 3.3.6 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hadoop Name: hadoop-mapreduce-client-common Version: 3.3.6 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hadoop Name: hadoop-mapreduce-client-core Version: 3.3.6 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hadoop Name: hadoop-mapreduce-client-jobclient Version: 3.3.6 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hadoop Name: hadoop-yarn-api Version: 3.3.6 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hadoop Name: hadoop-yarn-client Version: 3.3.6 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hadoop Name: hadoop-yarn-common Version: 3.3.6 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hadoop.thirdparty Name: hadoop-shaded-guava Version: 1.2.0 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hadoop.thirdparty Name: hadoop-shaded-protobuf_3_7 Version: 1.1.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hive Name: hive-common Version: 2.3.9 +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hive Name: hive-metastore Version: 2.3.9 +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hive Name: hive-serde Version: 2.3.9 +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hive Name: hive-shims Version: 2.3.9 +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hive Name: hive-storage-api Version: 2.4.0 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hive.shims Name: hive-shims-0.23 Version: 2.3.9 +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hive.shims Name: hive-shims-common Version: 2.3.9 +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hive.shims Name: hive-shims-scheduler Version: 2.3.9 +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.httpcomponents Name: httpclient Version: 4.5.13 +Project URL (from POM): http://hc.apache.org/httpcomponents-client +License (from POM): Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.httpcomponents Name: httpcore Version: 4.4.16 +Project URL (from POM): http://hc.apache.org/httpcomponents-core-ga +License (from POM): Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.httpcomponents.client5 Name: httpclient5 Version: 5.3.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.httpcomponents.core5 Name: httpcore5 Version: 5.2.4 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.httpcomponents.core5 Name: httpcore5-h2 Version: 5.2.4 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-admin Version: 1.0.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-client Version: 1.0.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-common Version: 1.0.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-core Version: 1.0.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-crypto Version: 1.0.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-identity Version: 1.0.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-server Version: 1.0.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-simplekdc Version: 1.0.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-util Version: 1.0.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerby-asn1 Version: 1.0.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerby-config Version: 1.0.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerby-pkix Version: 1.0.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerby-util Version: 1.0.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerby-xdr Version: 1.0.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: token-provider Version: 1.0.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.orc Name: orc-core Version: 1.9.4 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.orc Name: orc-shims Version: 1.9.4 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.parquet Name: parquet-avro Version: 1.13.1 +Project URL (from POM): https://parquet.apache.org +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.parquet Name: parquet-column Version: 1.13.1 +Project URL (from POM): https://parquet.apache.org +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.parquet Name: parquet-common Version: 1.13.1 +Project URL (from POM): https://parquet.apache.org +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.parquet Name: parquet-encoding Version: 1.13.1 +Project URL (from POM): https://parquet.apache.org +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.parquet Name: parquet-format-structures Version: 1.13.1 +Project URL (from POM): https://parquet.apache.org/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.parquet Name: parquet-hadoop Version: 1.13.1 +Project URL (from POM): https://parquet.apache.org +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.parquet Name: parquet-jackson Version: 1.13.1 +Project URL (from POM): https://parquet.apache.org +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.thrift Name: libfb303 Version: 0.9.3 +Project URL (from POM): http://thrift.apache.org +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.thrift Name: libthrift Version: 0.9.3 +Project URL (from POM): http://thrift.apache.org +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.yetus Name: audience-annotations Version: 0.13.0 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.zookeeper Name: zookeeper Version: 3.6.3 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.zookeeper Name: zookeeper-jute Version: 3.6.3 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.checkerframework Name: checker-qual Version: 3.44.0 +Manifest License: MIT (Not packaged) +Project URL (from POM): https://checkerframework.org/ +License (from POM): The MIT License - http://opensource.org/licenses/MIT + +-------------------------------------------------------------------------------- + +Group: org.codehaus.jettison Name: jettison Version: 1.5.4 +Project URL (from POM): https://github.com/jettison-json/jettison +License (from POM): Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: org.codehaus.mojo Name: animal-sniffer-annotations Version: 1.23 +License (from POM): MIT license - https://spdx.org/licenses/MIT.txt +License (from POM): The Apache Software License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.codehaus.woodstox Name: stax2-api Version: 4.2.2 +Project URL (from manifest): https://github.com/FasterXML/stax2-api +Project URL (from POM): https://github.com/FasterXML/stax2-api +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The BSD 2-Clause License - http://www.opensource.org/licenses/bsd-license.php + +-------------------------------------------------------------------------------- + +Group: org.conscrypt Name: conscrypt-openjdk-uber Version: 2.5.2 +Project URL (from POM): https://conscrypt.org/ +License (from POM): Apache 2 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: org.datanucleus Name: datanucleus-api-jdo Version: 4.2.4 +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.datanucleus Name: datanucleus-core Version: 4.1.17 +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.datanucleus Name: datanucleus-rdbms Version: 4.1.19 +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.datanucleus Name: javax.jdo Version: 3.2.0-m3 +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty Name: jetty-client Version: 9.4.51.v20230217 +Project URL (from manifest): https://eclipse.org/jetty +License (from POM): Apache Software License - Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0 +License (from POM): Eclipse Public License - Version 1.0 - https://www.eclipse.org/org/documents/epl-v10.php + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty Name: jetty-http Version: 9.4.51.v20230217 +Project URL (from manifest): https://eclipse.org/jetty +License (from POM): Apache Software License - Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0 +License (from POM): Eclipse Public License - Version 1.0 - https://www.eclipse.org/org/documents/epl-v10.php + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty Name: jetty-io Version: 9.4.51.v20230217 +Project URL (from manifest): https://eclipse.org/jetty +License (from POM): Apache Software License - Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0 +License (from POM): Eclipse Public License - Version 1.0 - https://www.eclipse.org/org/documents/epl-v10.php + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty Name: jetty-security Version: 9.4.51.v20230217 +Project URL (from manifest): https://eclipse.org/jetty +License (from POM): Apache Software License - Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0 +License (from POM): Eclipse Public License - Version 1.0 - https://www.eclipse.org/org/documents/epl-v10.php + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty Name: jetty-servlet Version: 9.4.51.v20230217 +Project URL (from manifest): https://eclipse.org/jetty +License (from POM): Apache Software License - Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0 +License (from POM): Eclipse Public License - Version 1.0 - https://www.eclipse.org/org/documents/epl-v10.php + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty Name: jetty-util Version: 9.4.51.v20230217 +Project URL (from manifest): https://eclipse.org/jetty +License (from POM): Apache Software License - Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0 +License (from POM): Eclipse Public License - Version 1.0 - https://www.eclipse.org/org/documents/epl-v10.php + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty Name: jetty-util-ajax Version: 9.4.51.v20230217 +Project URL (from manifest): https://eclipse.org/jetty +License (from POM): Apache Software License - Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0 +License (from POM): Eclipse Public License - Version 1.0 - https://www.eclipse.org/org/documents/epl-v10.php + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty Name: jetty-webapp Version: 9.4.51.v20230217 +Project URL (from manifest): https://eclipse.org/jetty +License (from POM): Apache Software License - Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0 +License (from POM): Eclipse Public License - Version 1.0 - https://www.eclipse.org/org/documents/epl-v10.php + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty Name: jetty-xml Version: 9.4.51.v20230217 +Project URL (from manifest): https://eclipse.org/jetty +License (from POM): Apache Software License - Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0 +License (from POM): Eclipse Public License - Version 1.0 - https://www.eclipse.org/org/documents/epl-v10.php + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty.websocket Name: websocket-api Version: 9.4.51.v20230217 +Project URL (from manifest): https://eclipse.org/jetty +License (from POM): Apache Software License - Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0 +License (from POM): Eclipse Public License - Version 1.0 - https://www.eclipse.org/org/documents/epl-v10.php + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty.websocket Name: websocket-client Version: 9.4.51.v20230217 +Project URL (from manifest): https://eclipse.org/jetty +License (from POM): Apache Software License - Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0 +License (from POM): Eclipse Public License - Version 1.0 - https://www.eclipse.org/org/documents/epl-v10.php + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty.websocket Name: websocket-common Version: 9.4.51.v20230217 +Project URL (from manifest): https://eclipse.org/jetty +License (from POM): Apache Software License - Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0 +License (from POM): Eclipse Public License - Version 1.0 - https://www.eclipse.org/org/documents/epl-v10.php + +-------------------------------------------------------------------------------- + +Group: org.jetbrains Name: annotations Version: 17.0.0 +Project URL (from POM): https://github.com/JetBrains/java-annotations +License (from POM): The Apache Software License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.jetbrains.kotlin Name: kotlin-stdlib Version: 1.4.10 +Project URL (from POM): https://kotlinlang.org/ +License (from POM): The Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.jetbrains.kotlin Name: kotlin-stdlib-common Version: 1.4.10 +Project URL (from POM): https://kotlinlang.org/ +License (from POM): The Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.jline Name: jline Version: 3.9.0 +License (from POM): The BSD License - http://www.opensource.org/licenses/bsd-license.php + +-------------------------------------------------------------------------------- + +Group: org.ow2.asm Name: asm Version: 9.3 +Project URL (from manifest): http://asm.ow2.org +Manifest License: BSD-3-Clause;link=https://asm.ow2.io/LICENSE.txt (Not packaged) +Project URL (from POM): http://asm.ow2.io/ +License (from POM): BSD-3-Clause - https://asm.ow2.io/license.html +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.reactivestreams Name: reactive-streams Version: 1.0.4 +Project URL (from manifest): http://reactive-streams.org +Project URL (from POM): http://www.reactive-streams.org/ +License (from POM): MIT-0 - https://spdx.org/licenses/MIT-0.html + +-------------------------------------------------------------------------------- + +Group: org.roaringbitmap Name: RoaringBitmap Version: 1.2.1 +Project URL (from POM): https://github.com/RoaringBitmap/RoaringBitmap +License (from POM): Apache 2 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.slf4j Name: slf4j-api Version: 2.0.13 +Project URL (from manifest): http://www.slf4j.org +Project URL (from POM): http://www.slf4j.org +License (from POM): MIT License - http://www.opensource.org/licenses/mit-license.php + +-------------------------------------------------------------------------------- + +Group: org.slf4j Name: slf4j-reload4j Version: 1.7.36 +Project URL (from POM): http://reload4j.qos.ch +License (from POM): MIT License - http://www.opensource.org/licenses/mit-license.php + +-------------------------------------------------------------------------------- + +Group: org.threeten Name: threeten-extra Version: 1.7.1 +Project URL (from manifest): https://www.threeten.org +Project URL (from POM): https://www.threeten.org/threeten-extra +License (from POM): BSD 3-clause - https://raw.githubusercontent.com/ThreeTen/threeten-extra/master/LICENSE.txt + +-------------------------------------------------------------------------------- + +Group: org.threeten Name: threetenbp Version: 1.6.9 +Project URL (from manifest): https://www.threeten.org +Project URL (from POM): https://www.threeten.org/threetenbp +License (from POM): BSD-3-Clause - https://raw.githubusercontent.com/ThreeTen/threetenbp/main/LICENSE.txt + +-------------------------------------------------------------------------------- + +Group: org.xerial.snappy Name: snappy-java Version: 1.1.10.5 +Project URL (from manifest): http://www.xerial.org/ +Project URL (from POM): https://github.com/xerial/snappy-java +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.html + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: annotations Version: 2.26.29 +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: apache-client Version: 2.26.29 +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: arns Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: auth Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: aws-core Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: aws-json-protocol Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: aws-query-protocol Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: aws-xml-protocol Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: checksums Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: checksums-spi Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: crt-core Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: dynamodb Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: endpoints-spi Version: 2.26.29 +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: glue Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: http-auth Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: http-auth-aws Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: http-auth-aws-eventstream Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: http-auth-spi Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: http-client-spi Version: 2.26.29 +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: iam Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: identity-spi Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: json-utils Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: kms Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: lakeformation Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: metrics-spi Version: 2.26.29 +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: netty-nio-client Version: 2.26.29 +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: profiles Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: protocol-core Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: regions Version: 2.26.29 +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: retries Version: 2.26.29 +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: retries-spi Version: 2.26.29 +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: s3 Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: sdk-core Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: sso Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: sts Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: third-party-jackson-core Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: utils Version: 2.26.29 +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.eventstream Name: eventstream Version: 1.0.1 +Project URL (from POM): https://github.com/awslabs/aws-eventstream-java +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- diff --git a/kafka-connect/kafka-connect-runtime/NOTICE b/kafka-connect/kafka-connect-runtime/NOTICE new file mode 100644 index 000000000000..aad25386f10b --- /dev/null +++ b/kafka-connect/kafka-connect-runtime/NOTICE @@ -0,0 +1,1723 @@ + +Apache Iceberg +Copyright 2017-2024 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +-------------------------------------------------------------------------------- + +This project includes code from Kite, developed at Cloudera, Inc. with +the following copyright notice: + +| Copyright 2013 Cloudera Inc. +| +| Licensed under the Apache License, Version 2.0 (the "License"); +| you may not use this file except in compliance with the License. +| You may obtain a copy of the License at +| +| http://www.apache.org/licenses/LICENSE-2.0 +| +| Unless required by applicable law or agreed to in writing, software +| distributed under the License is distributed on an "AS IS" BASIS, +| WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +| See the License for the specific language governing permissions and +| limitations under the License. + +-------------------------------------------------------------------------------- + +This binary artifact contains code from the following projects: + +-------------------------------------------------------------------------------- + +Group: org.apache.commons Name: commons-math3 Version: 3.1.1 + +Notice: Apache Commons Math +Copyright 2001-2012 The Apache Software Foundation + +This product includes software developed by +The Apache Software Foundation (http://www.apache.org/). + +=============================================================================== + +The BracketFinder (package org.apache.commons.math3.optimization.univariate) +and PowellOptimizer (package org.apache.commons.math3.optimization.general) +classes are based on the Python code in module "optimize.py" (version 0.5) +developed by Travis E. Oliphant for the SciPy library (http://www.scipy.org/) +Copyright © 2003-2009 SciPy Developers. +=============================================================================== + +The LinearConstraint, LinearObjectiveFunction, LinearOptimizer, +RelationShip, SimplexSolver and SimplexTableau classes in package +org.apache.commons.math3.optimization.linear include software developed by +Benjamin McCann (http://www.benmccann.com) and distributed with +the following copyright: Copyright 2009 Google Inc. +=============================================================================== + +This product includes software developed by the +University of Chicago, as Operator of Argonne National +Laboratory. +The LevenbergMarquardtOptimizer class in package +org.apache.commons.math3.optimization.general includes software +translated from the lmder, lmpar and qrsolv Fortran routines +from the Minpack package +Minpack Copyright Notice (1999) University of Chicago. All rights reserved +=============================================================================== + +The GraggBulirschStoerIntegrator class in package +org.apache.commons.math3.ode.nonstiff includes software translated +from the odex Fortran routine developed by E. Hairer and G. Wanner. +Original source copyright: +Copyright (c) 2004, Ernst Hairer +=============================================================================== + +The EigenDecompositionImpl class in package +org.apache.commons.math3.linear includes software translated +from some LAPACK Fortran routines. Original source copyright: +Copyright (c) 1992-2008 The University of Tennessee. All rights reserved. +=============================================================================== + +The MersenneTwister class in package org.apache.commons.math3.random +includes software translated from the 2002-01-26 version of +the Mersenne-Twister generator written in C by Makoto Matsumoto and Takuji +Nishimura. Original source copyright: +Copyright (C) 1997 - 2002, Makoto Matsumoto and Takuji Nishimura, +All rights reserved +=============================================================================== + +The LocalizedFormatsTest class in the unit tests is an adapted version of +the OrekitMessagesTest class from the orekit library distributed under the +terms of the Apache 2 licence. Original source copyright: +Copyright 2010 CS Systèmes d'Information +=============================================================================== + +The HermiteInterpolator class and its corresponding test have been imported from +the orekit library distributed under the terms of the Apache 2 licence. Original +source copyright: +Copyright 2010-2012 CS Systèmes d'Information +=============================================================================== + +The creation of the package "o.a.c.m.analysis.integration.gauss" was inspired +by an original code donated by Sébastien Brisard. +=============================================================================== + + +The complete text of licenses and disclaimers associated with the the original +sources enumerated above at the time of code translation are in the LICENSE.txt +file. + + +-------------------------------------------------------------------------------- + +Group: org.apache.httpcomponents.core5 Name: httpcore5-h2 Version: 5.2.4 + +Notice: Apache HttpComponents Core HTTP/2 +Copyright 2005-2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-admin Version: 1.0.1 + +Notice: Kerby-kerb Admin +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-crypto Version: 1.0.1 + +Notice: Kerby-kerb Crypto +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.orc Name: orc-core Version: 1.9.4 + +Notice: ORC Core +Copyright 2013-2024 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.hive Name: hive-shims Version: 2.3.9 + +Notice: Hive Shims +Copyright 2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerby-pkix Version: 1.0.1 + +Notice: Kerby PKIX Project +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.hive.shims Name: hive-shims-scheduler Version: 2.3.9 + +Notice: Hive Shims Scheduler +Copyright 2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.ant Name: ant Version: 1.9.1 +Group: org.apache.ant Name: ant-launcher Version: 1.9.1 + +Notice: Apache Ant + Copyright 1999-2013 The Apache Software Foundation + + The task is based on code Copyright (c) 2002, Landmark + Graphics Corp that has been kindly donated to the Apache Software + Foundation. + + +-------------------------------------------------------------------------------- + +Group: commons-lang Name: commons-lang Version: 2.6 + +Notice: Apache Commons Lang +Copyright 2001-2011 The Apache Software Foundation + +This product includes software developed by +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.commons Name: commons-configuration2 Version: 2.8.0 + +Notice: Apache Commons Configuration +Copyright 2001-2022 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerby-config Version: 1.0.1 + +Notice: Kerby Config +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.parquet Name: parquet-avro Version: 1.13.1 + +Notice: Apache Parquet MR (Incubating) +Copyright 2014-2015 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +-------------------------------------------------------------------------------- + +This product includes code from Apache Avro, which includes the following in +its NOTICE file: + + Apache Avro + Copyright 2010-2015 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-netty-shaded Version: 1.62.2 + +Notice: The Netty Project + ================= + +Please visit the Netty web site for more information: + + * http://netty.io/ + +Copyright 2016 The Netty Project + +The Netty Project licenses this file to you under the Apache License, +version 2.0 (the "License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at: + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, WITHOUT +WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the +License for the specific language governing permissions and limitations +under the License. + +------------------------------------------------------------------------------- +This product contains a forked and modified version of Tomcat Native + + * LICENSE: + * license/LICENSE.tomcat-native.txt (Apache License 2.0) + * HOMEPAGE: + * http://tomcat.apache.org/native-doc/ + * https://svn.apache.org/repos/asf/tomcat/native/ + +This product contains the Maven wrapper scripts from 'Maven Wrapper', that provides an easy way to ensure a user has everything necessary to run the Maven build. + + * LICENSE: + * license/LICENSE.mvn-wrapper.txt (Apache License 2.0) + * HOMEPAGE: + * https://github.com/takari/maven-wrapper + +This product contains small piece of code to support AIX, taken from netbsd. + + * LICENSE: + * license/LICENSE.aix-netbsd.txt (OpenSSL License) + * HOMEPAGE: + * https://ftp.netbsd.org/pub/NetBSD/NetBSD-current/src/crypto/external/bsd/openssl/dist + + +This product contains code from boringssl. + + * LICENSE (Combination ISC and OpenSSL license) + * license/LICENSE.boringssl.txt (Combination ISC and OpenSSL license) + * HOMEPAGE: + * https://boringssl.googlesource.com/boringssl/ + + +-------------------------------------------------------------------------------- + +Group: org.apache.curator Name: curator-recipes Version: 5.2.0 + +Notice: Curator Recipes +Copyright 2011-2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.hive Name: hive-storage-api Version: 2.4.0 + +Notice: Hive Storage API +Copyright 2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.hadoop Name: hadoop-annotations Version: 3.3.6 +Group: org.apache.hadoop Name: hadoop-auth Version: 3.3.6 +Group: org.apache.hadoop Name: hadoop-client Version: 3.3.6 +Group: org.apache.hadoop Name: hadoop-common Version: 3.3.6 +Group: org.apache.hadoop Name: hadoop-hdfs-client Version: 3.3.6 +Group: org.apache.hadoop Name: hadoop-mapreduce-client-common Version: 3.3.6 +Group: org.apache.hadoop Name: hadoop-mapreduce-client-core Version: 3.3.6 +Group: org.apache.hadoop Name: hadoop-mapreduce-client-jobclient Version: 3.3.6 +Group: org.apache.hadoop Name: hadoop-yarn-api Version: 3.3.6 +Group: org.apache.hadoop Name: hadoop-yarn-client Version: 3.3.6 +Group: org.apache.hadoop Name: hadoop-yarn-common Version: 3.3.6 + +Notice: Apache Hadoop +Copyright 2006 and onwards The Apache Software Foundation. + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +Export Control Notice +--------------------- + +This distribution includes cryptographic software. The country in +which you currently reside may have restrictions on the import, +possession, use, and/or re-export to another country, of +encryption software. BEFORE using any encryption software, please +check your country's laws, regulations and policies concerning the +import, possession, or use, and re-export of encryption software, to +see if this is permitted. See for more +information. + +The U.S. Government Department of Commerce, Bureau of Industry and +Security (BIS), has classified this software as Export Commodity +Control Number (ECCN) 5D002.C.1, which includes information security +software using or performing cryptographic functions with asymmetric +algorithms. The form and manner of this Apache Software Foundation +distribution makes it eligible for export under the License Exception +ENC Technology Software Unrestricted (TSU) exception (see the BIS +Export Administration Regulations, Section 740.13) for both object +code and source code. + +The following provides more details on the included cryptographic software: + +This software uses the SSL libraries from the Jetty project written +by mortbay.org. +Hadoop Yarn Server Web Proxy uses the BouncyCastle Java +cryptography APIs written by the Legion of the Bouncy Castle Inc. + + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.jackson.dataformat Name: jackson-dataformat-xml Version: 2.17.2 + +Notice: # Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers. + +## Copyright + +Copyright 2007-, Tatu Saloranta (tatu.saloranta@iki.fi) + +## Licensing + +Jackson components are licensed under Apache (Software) License, version 2.0, +as per accompanying LICENSE file. + +## Credits + +A list of contributors may be found from CREDITS file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. + + +-------------------------------------------------------------------------------- + +Group: org.apache.hive Name: hive-common Version: 2.3.9 + +Notice: Hive Common +Copyright 2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-common Version: 1.0.1 + +Notice: Kerby-kerb Common +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerby-xdr Version: 1.0.1 + +Notice: Kerby XDR Project +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.parquet Name: parquet-jackson Version: 1.13.1 + +Notice: # Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers. + +## Licensing + +Jackson 2.x core and extension components are licensed under Apache License 2.0 +To find the details that apply to this artifact see the accompanying LICENSE file. + +## Credits + +A list of contributors may be found from CREDITS(-2.x) file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. + + +-------------------------------------------------------------------------------- + +Group: jakarta.activation Name: jakarta.activation-api Version: 1.2.2 + +Notice: # Notices for Jakarta Activation + +This content is produced and maintained by Jakarta Activation project. + +* Project home: https://projects.eclipse.org/projects/ee4j.jaf + +## Copyright + +All content is the property of the respective authors or their employers. For +more information regarding authorship of content, please consult the listed +source code repository logs. + +## Declared Project Licenses + +This program and the accompanying materials are made available under the terms +of the Eclipse Distribution License v. 1.0, +which is available at http://www.eclipse.org/org/documents/edl-v10.php. + +SPDX-License-Identifier: BSD-3-Clause + +## Source Code + +The project maintains the following source code repositories: + +* https://github.com/eclipse-ee4j/jaf + +## Third-party Content + +This project leverages the following third party content. + +JUnit (4.12) + +* License: Eclipse Public License + + +-------------------------------------------------------------------------------- + +Group: jakarta.xml.bind Name: jakarta.xml.bind-api Version: 2.3.3 + +Notice: [//]: # " Copyright (c) 2018, 2019 Oracle and/or its affiliates. All rights reserved. " +[//]: # " " +[//]: # " This program and the accompanying materials are made available under the " +[//]: # " terms of the Eclipse Distribution License v. 1.0, which is available at " +[//]: # " http://www.eclipse.org/org/documents/edl-v10.php. " +[//]: # " " +[//]: # " SPDX-License-Identifier: BSD-3-Clause " + +# Notices for Jakarta XML Binding + +This content is produced and maintained by the Jakarta XML Binding +project. + +* Project home: https://projects.eclipse.org/projects/ee4j.jaxb + +## Trademarks + +Jakarta XML Binding is a trademark of the Eclipse Foundation. + +## Copyright + +All content is the property of the respective authors or their employers. For +more information regarding authorship of content, please consult the listed +source code repository logs. + +## Declared Project Licenses + +This program and the accompanying materials are made available under the terms +of the Eclipse Distribution License v. 1.0 which is available at +http://www.eclipse.org/org/documents/edl-v10.php. + +SPDX-License-Identifier: BSD-3-Clause + +## Source Code + +The project maintains the following source code repositories: + +* https://github.com/eclipse-ee4j/jaxb-api +* https://github.com/eclipse-ee4j/jaxb-tck + +## Third-party Content + +This project leverages the following third party content. + +Apache River (3.0.0) + +* License: Apache-2.0 AND BSD-3-Clause + +ASM 7 (n/a) + +* License: BSD-3-Clause +* Project: https://asm.ow2.io/ +* Source: + https://repository.ow2.org/nexus/#nexus-search;gav~org.ow2.asm~asm-commons~~~~kw,versionexpand + +JTHarness (5.0) + +* License: (GPL-2.0 OR GPL-2.0 WITH Classpath-exception-2.0) +* Project: https://wiki.openjdk.java.net/display/CodeTools/JT+Harness +* Source: http://hg.openjdk.java.net/code-tools/jtharness/ + +normalize.css (3.0.2) + +* License: MIT + +SigTest (n/a) + +* License: GPL-2.0 OR GPL-2.0 WITH Classpath-exception-2.0 + +## Cryptography + +Content may contain encryption software. The country in which you are currently +may have restrictions on the import, possession, and use, and/or re-export to +another country, of encryption software. BEFORE using any encryption software, +please check the country's laws, regulations and policies concerning the import, +possession, or use, and re-export of encryption software, to see if this is +permitted. + + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.jackson.module Name: jackson-module-jaxb-annotations Version: 2.17.2 + +Notice: # Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers, as well as supported +commercially by FasterXML.com. + +## Licensing + +Jackson core and extension components may licensed under different licenses. +To find the details that apply to this artifact see the accompanying LICENSE file. +For more information, including possible other licensing options, contact +FasterXML.com (http://fasterxml.com). + +## Credits + +A list of contributors may be found from CREDITS file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. + + +-------------------------------------------------------------------------------- + +Group: org.apache.commons Name: commons-compress Version: 1.26.2 + +Notice: Apache Commons Compress +Copyright 2002-2024 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (https://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-client Version: 1.0.1 + +Notice: Kerby-kerb Client +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: commons-io Name: commons-io Version: 2.16.1 + +Notice: Apache Commons IO +Copyright 2002-2024 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (https://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.datanucleus Name: datanucleus-core Version: 4.1.17 + +Notice: ========================================================================= +== NOTICE file corresponding to section 4(d) of the Apache License, == +== Version 2.0, in this case for the DataNucleus distribution. == +========================================================================= + +=================================================================== +This product includes software developed by many individuals, +including the following: +=================================================================== +Erik Bengtson +Andy Jefferson + + +=================================================================== +This product has included contributions from some individuals, +including the following: +=================================================================== +Joerg von Frantzius +Thomas Marti +Barry Haddow +Marco Schulze +Ralph Ullrich +David Ezzio +Brendan de Beer +David Eaves +Martin Taal +Tony Lai +Roland Szabo +Marcus Mennemeier +Xuan Baldauf +Eric Sultan + + +=================================================================== +This product also includes software developed by the TJDO project +(http://tjdo.sourceforge.net/). +=================================================================== + + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: annotations Version: 2.26.29 +Group: software.amazon.awssdk Name: apache-client Version: 2.26.29 +Group: software.amazon.awssdk Name: arns Version: 2.26.29 +Group: software.amazon.awssdk Name: auth Version: 2.26.29 +Group: software.amazon.awssdk Name: aws-core Version: 2.26.29 +Group: software.amazon.awssdk Name: aws-json-protocol Version: 2.26.29 +Group: software.amazon.awssdk Name: aws-query-protocol Version: 2.26.29 +Group: software.amazon.awssdk Name: aws-xml-protocol Version: 2.26.29 +Group: software.amazon.awssdk Name: checksums Version: 2.26.29 +Group: software.amazon.awssdk Name: checksums-spi Version: 2.26.29 +Group: software.amazon.awssdk Name: crt-core Version: 2.26.29 +Group: software.amazon.awssdk Name: dynamodb Version: 2.26.29 +Group: software.amazon.awssdk Name: endpoints-spi Version: 2.26.29 +Group: software.amazon.awssdk Name: glue Version: 2.26.29 +Group: software.amazon.awssdk Name: http-auth Version: 2.26.29 +Group: software.amazon.awssdk Name: http-auth-aws Version: 2.26.29 +Group: software.amazon.awssdk Name: http-auth-aws-eventstream Version: 2.26.29 +Group: software.amazon.awssdk Name: http-auth-spi Version: 2.26.29 +Group: software.amazon.awssdk Name: http-client-spi Version: 2.26.29 +Group: software.amazon.awssdk Name: iam Version: 2.26.29 +Group: software.amazon.awssdk Name: identity-spi Version: 2.26.29 +Group: software.amazon.awssdk Name: json-utils Version: 2.26.29 +Group: software.amazon.awssdk Name: kms Version: 2.26.29 +Group: software.amazon.awssdk Name: lakeformation Version: 2.26.29 +Group: software.amazon.awssdk Name: metrics-spi Version: 2.26.29 +Group: software.amazon.awssdk Name: netty-nio-client Version: 2.26.29 +Group: software.amazon.awssdk Name: profiles Version: 2.26.29 +Group: software.amazon.awssdk Name: protocol-core Version: 2.26.29 +Group: software.amazon.awssdk Name: regions Version: 2.26.29 +Group: software.amazon.awssdk Name: retries Version: 2.26.29 +Group: software.amazon.awssdk Name: retries-spi Version: 2.26.29 +Group: software.amazon.awssdk Name: s3 Version: 2.26.29 +Group: software.amazon.awssdk Name: sdk-core Version: 2.26.29 +Group: software.amazon.awssdk Name: sso Version: 2.26.29 +Group: software.amazon.awssdk Name: sts Version: 2.26.29 +Group: software.amazon.awssdk Name: utils Version: 2.26.29 + +Notice: AWS SDK for Java 2.0 +Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved. + +This product includes software developed by +Amazon Technologies, Inc (http://www.amazon.com/). + +********************** +THIRD PARTY COMPONENTS +********************** +This software includes third party software subject to the following copyrights: +- XML parsing and utility functions from JetS3t - Copyright 2006-2009 James Murty. +- PKCS#1 PEM encoded private key parsing and utility functions from oauth.googlecode.com - Copyright 1998-2010 AOL Inc. +- Apache Commons Lang - https://github.com/apache/commons-lang +- Netty Reactive Streams - https://github.com/playframework/netty-reactive-streams +- Jackson-core - https://github.com/FasterXML/jackson-core +- Jackson-dataformat-cbor - https://github.com/FasterXML/jackson-dataformats-binary + +The licenses for these third party components are included in LICENSE.txt + +- For Apache Commons Lang see also this required NOTICE: + Apache Commons Lang + Copyright 2001-2020 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (https://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.httpcomponents.client5 Name: httpclient5 Version: 5.3.1 + +Notice: Apache HttpClient +Copyright 1999-2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.hive.shims Name: hive-shims-common Version: 2.3.9 + +Notice: Hive Shims Common +Copyright 2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: commons-logging Name: commons-logging Version: 1.2 + +Notice: Apache Commons Logging +Copyright 2003-2014 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: commons-dbcp Name: commons-dbcp Version: 1.4 + +Notice: Apache Commons DBCP +Copyright 2001-2010 The Apache Software Foundation + +This product includes software developed by +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: commons-net Name: commons-net Version: 3.9.0 + +Notice: Apache Commons Net +Copyright 2001-2022 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (https://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: com.squareup.okhttp3 Name: okhttp Version: 4.9.3 + +Notice: Note that publicsuffixes.gz is compiled from The Public Suffix List: +https://publicsuffix.org/list/public_suffix_list.dat + +It is subject to the terms of the Mozilla Public License, v. 2.0: +https://mozilla.org/MPL/2.0/ + + +-------------------------------------------------------------------------------- + +Group: commons-collections Name: commons-collections Version: 3.2.2 + +Notice: Apache Commons Collections +Copyright 2001-2015 The Apache Software Foundation + +This product includes software developed by +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: jakarta.activation Name: jakarta.activation-api Version: 1.2.1 + +Notice: # Notices for Eclipse Project for JAF + +This content is produced and maintained by the Eclipse Project for JAF project. + +* Project home: https://projects.eclipse.org/projects/ee4j.jaf + +## Copyright + +All content is the property of the respective authors or their employers. For +more information regarding authorship of content, please consult the listed +source code repository logs. + +## Declared Project Licenses + +This program and the accompanying materials are made available under the terms +of the Eclipse Distribution License v. 1.0, +which is available at http://www.eclipse.org/org/documents/edl-v10.php. + +SPDX-License-Identifier: BSD-3-Clause + +## Source Code + +The project maintains the following source code repositories: + +* https://github.com/eclipse-ee4j/jaf + +## Third-party Content + +This project leverages the following third party content. + +JUnit (4.12) + +* License: Eclipse Public License + + +-------------------------------------------------------------------------------- + +Group: org.apache.derby Name: derby Version: 10.10.2.0 + +Notice: ========================================================================= +== NOTICE file corresponding to section 4(d) of the Apache License, +== Version 2.0, in this case for the Apache Derby distribution. +== +== DO NOT EDIT THIS FILE DIRECTLY. IT IS GENERATED +== BY THE buildnotice TARGET IN THE TOP LEVEL build.xml FILE. +== +========================================================================= + +Apache Derby +Copyright 2004-2014 The Apache Software Foundation + +This product includes software developed by +The Apache Software Foundation (http://www.apache.org/). + + +========================================================================= + +Portions of Derby were originally developed by +International Business Machines Corporation and are +licensed to the Apache Software Foundation under the +"Software Grant and Corporate Contribution License Agreement", +informally known as the "Derby CLA". +The following copyright notice(s) were affixed to portions of the code +with which this file is now or was at one time distributed +and are placed here unaltered. + +(C) Copyright 1997,2004 International Business Machines Corporation. All rights reserved. + +(C) Copyright IBM Corp. 2003. + + +========================================================================= + + +The portion of the functionTests under 'nist' was originally +developed by the National Institute of Standards and Technology (NIST), +an agency of the United States Department of Commerce, and adapted by +International Business Machines Corporation in accordance with the NIST +Software Acknowledgment and Redistribution document at +http://www.itl.nist.gov/div897/ctg/sql_form.htm + + + +========================================================================= + + +The JDBC apis for small devices and JDBC3 (under java/stubs/jsr169 and +java/stubs/jdbc3) were produced by trimming sources supplied by the +Apache Harmony project. In addition, the Harmony SerialBlob and +SerialClob implementations are used. The following notice covers the Harmony sources: + +Portions of Harmony were originally developed by +Intel Corporation and are licensed to the Apache Software +Foundation under the "Software Grant and Corporate Contribution +License Agreement", informally known as the "Intel Harmony CLA". + + +========================================================================= + + +The Derby build relies on source files supplied by the Apache Felix +project. The following notice covers the Felix files: + + Apache Felix Main + Copyright 2008 The Apache Software Foundation + + + I. Included Software + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + Licensed under the Apache License 2.0. + + This product includes software developed at + The OSGi Alliance (http://www.osgi.org/). + Copyright (c) OSGi Alliance (2000, 2007). + Licensed under the Apache License 2.0. + + This product includes software from http://kxml.sourceforge.net. + Copyright (c) 2002,2003, Stefan Haustein, Oberhausen, Rhld., Germany. + Licensed under BSD License. + + II. Used Software + + This product uses software developed at + The OSGi Alliance (http://www.osgi.org/). + Copyright (c) OSGi Alliance (2000, 2007). + Licensed under the Apache License 2.0. + + + III. License Summary + - Apache License 2.0 + - BSD License + + +========================================================================= + + +The Derby build relies on jar files supplied by the Apache Xalan +project. The following notice covers the Xalan jar files: + + ========================================================================= + == NOTICE file corresponding to section 4(d) of the Apache License, == + == Version 2.0, in this case for the Apache Xalan Java distribution. == + ========================================================================= + + Apache Xalan (Xalan XSLT processor) + Copyright 1999-2006 The Apache Software Foundation + + Apache Xalan (Xalan serializer) + Copyright 1999-2006 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + ========================================================================= + Portions of this software was originally based on the following: + - software copyright (c) 1999-2002, Lotus Development Corporation., + http://www.lotus.com. + - software copyright (c) 2001-2002, Sun Microsystems., + http://www.sun.com. + - software copyright (c) 2003, IBM Corporation., + http://www.ibm.com. + + ========================================================================= + The binary distribution package (ie. jars, samples and documentation) of + this product includes software developed by the following: + + - The Apache Software Foundation + - Xerces Java - see LICENSE.txt + - JAXP 1.3 APIs - see LICENSE.txt + - Bytecode Engineering Library - see LICENSE.txt + - Regular Expression - see LICENSE.txt + + - Scott Hudson, Frank Flannery, C. Scott Ananian + - CUP Parser Generator runtime (javacup\runtime) - see LICENSE.txt + + ========================================================================= + The source distribution package (ie. all source and tools required to build + Xalan Java) of this product includes software developed by the following: + + - The Apache Software Foundation + - Xerces Java - see LICENSE.txt + - JAXP 1.3 APIs - see LICENSE.txt + - Bytecode Engineering Library - see LICENSE.txt + - Regular Expression - see LICENSE.txt + - Ant - see LICENSE.txt + - Stylebook doc tool - see LICENSE.txt + + - Elliot Joel Berk and C. Scott Ananian + - Lexical Analyzer Generator (JLex) - see LICENSE.txt + + ========================================================================= + Apache Xerces Java + Copyright 1999-2006 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + Portions of Apache Xerces Java in xercesImpl.jar and xml-apis.jar + were originally based on the following: + - software copyright (c) 1999, IBM Corporation., http://www.ibm.com. + - software copyright (c) 1999, Sun Microsystems., http://www.sun.com. + - voluntary contributions made by Paul Eng on behalf of the + Apache Software Foundation that were originally developed at iClick, Inc., + software copyright (c) 1999. + + ========================================================================= + Apache xml-commons xml-apis (redistribution of xml-apis.jar) + + Apache XML Commons + Copyright 2001-2003,2006 The Apache Software Foundation. + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + Portions of this software were originally based on the following: + - software copyright (c) 1999, IBM Corporation., http://www.ibm.com. + - software copyright (c) 1999, Sun Microsystems., http://www.sun.com. + - software copyright (c) 2000 World Wide Web Consortium, http://www.w3.org + + +-------------------------------------------------------------------------------- + +Group: com.google.inject Name: guice Version: 4.0 + +Notice: Google Guice - Core Library +Copyright 2006-2015 Google, Inc. + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: ch.qos.reload4j Name: reload4j Version: 1.2.22 +Group: log4j Name: log4j Version: 1.2.17 + +Notice: Apache log4j +Copyright 2007 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: commons-codec Name: commons-codec Version: 1.17.1 + +Notice: Apache Commons Codec +Copyright 2002-2024 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (https://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: joda-time Name: joda-time Version: 2.8.1 + +Notice: ============================================================================= += NOTICE file corresponding to section 4d of the Apache License Version 2.0 = +============================================================================= +This product includes software developed by +Joda.org (http://www.joda.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.commons Name: commons-text Version: 1.10.0 + +Notice: Apache Commons Text +Copyright 2014-2022 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (https://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-identity Version: 1.0.1 + +Notice: Kerby-kerb Identity +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.curator Name: curator-client Version: 5.2.0 + +Notice: Curator Client +Copyright 2011-2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.thrift Name: libthrift Version: 0.9.3 + +Notice: Apache Thrift +Copyright 2006-2010 The Apache Software Foundation. + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.jackson.core Name: jackson-core Version: 2.17.2 +Group: software.amazon.awssdk Name: third-party-jackson-core Version: 2.26.29 + +Notice: # Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers. + +## Copyright + +Copyright 2007-, Tatu Saloranta (tatu.saloranta@iki.fi) + +## Licensing + +Jackson 2.x core and extension components are licensed under Apache License 2.0 +To find the details that apply to this artifact see the accompanying LICENSE file. + +## Credits + +A list of contributors may be found from CREDITS(-2.x) file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. + +## FastDoubleParser + +jackson-core bundles a shaded copy of FastDoubleParser . +That code is available under an MIT license +under the following copyright. + +Copyright © 2023 Werner Randelshofer, Switzerland. MIT License. + +See FastDoubleParser-NOTICE for details of other source code included in FastDoubleParser +and the licenses and copyrights that apply to that code. + + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.jackson.datatype Name: jackson-datatype-jsr310 Version: 2.17.2 + +Notice: # Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers. + +## Licensing + +Jackson components are licensed under Apache (Software) License, version 2.0, +as per accompanying LICENSE file. + +## Credits + +A list of contributors may be found from CREDITS file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. + + +-------------------------------------------------------------------------------- + +Group: org.apache.curator Name: curator-framework Version: 5.2.0 + +Notice: Curator Framework +Copyright 2011-2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.httpcomponents Name: httpclient Version: 4.5.13 + +Notice: Apache HttpClient +Copyright 1999-2020 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerby-util Version: 1.0.1 + +Notice: Kerby Util +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.hive Name: hive-serde Version: 2.3.9 + +Notice: Hive Serde +Copyright 2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.httpcomponents.core5 Name: httpcore5 Version: 5.2.4 + +Notice: Apache HttpComponents Core HTTP/1.1 +Copyright 2005-2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-core Version: 1.0.1 + +Notice: Kerby-kerb core +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.datanucleus Name: datanucleus-api-jdo Version: 4.2.4 + +Notice: ========================================================================= +== NOTICE file corresponding to section 4(d) of the Apache License, == +== Version 2.0, in this case for the DataNucleus distribution. == +========================================================================= + +=================================================================== +This product includes software developed by many individuals, +including the following: +=================================================================== +Erik Bengtson +Andy Jefferson + + +=================================================================== +This product has included contributions from some individuals, +including the following: +=================================================================== + + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-util Version: 1.0.1 + +Notice: Kerby-kerb Util +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.datanucleus Name: datanucleus-rdbms Version: 4.1.19 + +Notice: ========================================================================= +== NOTICE file corresponding to section 4(d) of the Apache License, == +== Version 2.0, in this case for the DataNucleus distribution. == +========================================================================= + +=================================================================== +This product includes software developed by many individuals, +including the following: +=================================================================== +Andy Jefferson +Erik Bengtson +Joerg von Frantzius +Marco Schulze + + +=================================================================== +This product has included contributions from some individuals, +including the following: +=================================================================== +Barry Haddow +Ralph Ullrich +David Ezzio +Brendan de Beer +David Eaves +Martin Taal +Tony Lai +Roland Szabo +Anton Troshin (Timesten) + + +=================================================================== +This product also includes software developed by the TJDO project +(http://tjdo.sourceforge.net/). +=================================================================== + +=================================================================== +This product also includes software developed by the Apache Commons project +(http://commons.apache.org/). +=================================================================== + + +-------------------------------------------------------------------------------- + +Group: org.apache.hive Name: hive-metastore Version: 2.3.9 + +Notice: Hive Metastore +Copyright 2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: commons-cli Name: commons-cli Version: 1.2 + +Notice: Apache Commons CLI +Copyright 2001-2009 The Apache Software Foundation + +This product includes software developed by +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: commons-pool Name: commons-pool Version: 1.6 + +Notice: Apache Commons Pool +Copyright 2001-2012 The Apache Software Foundation + +This product includes software developed by +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: javax.jdo Name: jdo-api Version: 3.0.1 + +Notice: Apache Java Data Objects (JDO) +Copyright 2005-2006 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.orc Name: orc-shims Version: 1.9.4 + +Notice: ORC Shims +Copyright 2013-2024 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.hadoop.thirdparty Name: hadoop-shaded-guava Version: 1.2.0 +Group: org.apache.hadoop.thirdparty Name: hadoop-shaded-protobuf_3_7 Version: 1.1.1 + +Notice: Apache Hadoop Third-party Libs +Copyright 2020 and onwards The Apache Software Foundation. + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.httpcomponents Name: httpcore Version: 4.4.16 + +Notice: Apache HttpCore +Copyright 2005-2022 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: commons-beanutils Name: commons-beanutils Version: 1.9.4 + +Notice: Apache Commons BeanUtils +Copyright 2000-2019 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.jackson.core Name: jackson-annotations Version: 2.17.2 +Group: com.fasterxml.jackson.core Name: jackson-databind Version: 2.17.2 + +Notice: # Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers. + +## Copyright + +Copyright 2007-, Tatu Saloranta (tatu.saloranta@iki.fi) + +## Licensing + +Jackson 2.x core and extension components are licensed under Apache License 2.0 +To find the details that apply to this artifact see the accompanying LICENSE file. + +## Credits + +A list of contributors may be found from CREDITS(-2.x) file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. + + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-simplekdc Version: 1.0.1 + +Notice: Kerb Simple Kdc +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-server Version: 1.0.1 + +Notice: Kerby-kerb Server +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.avro Name: avro Version: 1.12.0 + +Notice: Apache Avro +Copyright 2009-2024 The Apache Software Foundation + + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.hive.shims Name: hive-shims-0.23 Version: 2.3.9 + +Notice: Hive Shims 0.23 +Copyright 2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerby-asn1 Version: 1.0.1 + +Notice: Kerby ASN1 Project +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: token-provider Version: 1.0.1 + +Notice: Token provider +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.jackson.jaxrs Name: jackson-jaxrs-json-provider Version: 2.17.2 + +Notice: # Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers, as well as supported +commercially by FasterXML.com. + +## Licensing + +Jackson core and extension components may be licensed under different licenses. +To find the details that apply to this artifact see the accompanying LICENSE file. +For more information, including possible other licensing options, contact +FasterXML.com (http://fasterxml.com). + +## Credits + +A list of contributors may be found from CREDITS file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. + + +-------------------------------------------------------------------------------- + +Group: org.apache.yetus Name: audience-annotations Version: 0.13.0 + +Notice: Apache Yetus - Audience Annotations +Copyright 2015-2020 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.commons Name: commons-lang3 Version: 3.14.0 + +Notice: Apache Commons Lang +Copyright 2001-2023 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (https://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty Name: jetty-client Version: 9.4.51.v20230217 +Group: org.eclipse.jetty Name: jetty-http Version: 9.4.51.v20230217 +Group: org.eclipse.jetty Name: jetty-io Version: 9.4.51.v20230217 +Group: org.eclipse.jetty Name: jetty-security Version: 9.4.51.v20230217 +Group: org.eclipse.jetty Name: jetty-servlet Version: 9.4.51.v20230217 +Group: org.eclipse.jetty Name: jetty-util Version: 9.4.51.v20230217 +Group: org.eclipse.jetty Name: jetty-util-ajax Version: 9.4.51.v20230217 +Group: org.eclipse.jetty Name: jetty-webapp Version: 9.4.51.v20230217 +Group: org.eclipse.jetty Name: jetty-xml Version: 9.4.51.v20230217 +Group: org.eclipse.jetty.websocket Name: websocket-api Version: 9.4.51.v20230217 +Group: org.eclipse.jetty.websocket Name: websocket-client Version: 9.4.51.v20230217 +Group: org.eclipse.jetty.websocket Name: websocket-common Version: 9.4.51.v20230217 + +Notice: ============================================================== + Jetty Web Container + Copyright 1995-2018 Mort Bay Consulting Pty Ltd. +============================================================== + +The Jetty Web Container is Copyright Mort Bay Consulting Pty Ltd +unless otherwise noted. + +Jetty is dual licensed under both + + * The Apache 2.0 License + http://www.apache.org/licenses/LICENSE-2.0.html + + and + + * The Eclipse Public 1.0 License + http://www.eclipse.org/legal/epl-v10.html + +Jetty may be distributed under either license. + +------ +Eclipse + +The following artifacts are EPL. + * org.eclipse.jetty.orbit:org.eclipse.jdt.core + +The following artifacts are EPL and ASL2. + * org.eclipse.jetty.orbit:javax.security.auth.message + + +The following artifacts are EPL and CDDL 1.0. + * org.eclipse.jetty.orbit:javax.mail.glassfish + + +------ +Oracle + +The following artifacts are CDDL + GPLv2 with classpath exception. +https://glassfish.dev.java.net/nonav/public/CDDL+GPL.html + + * javax.servlet:javax.servlet-api + * javax.annotation:javax.annotation-api + * javax.transaction:javax.transaction-api + * javax.websocket:javax.websocket-api + +------ +Oracle OpenJDK + +If ALPN is used to negotiate HTTP/2 connections, then the following +artifacts may be included in the distribution or downloaded when ALPN +module is selected. + + * java.sun.security.ssl + +These artifacts replace/modify OpenJDK classes. The modififications +are hosted at github and both modified and original are under GPL v2 with +classpath exceptions. +http://openjdk.java.net/legal/gplv2+ce.html + + +------ +OW2 + +The following artifacts are licensed by the OW2 Foundation according to the +terms of http://asm.ow2.org/license.html + +org.ow2.asm:asm-commons +org.ow2.asm:asm + + +------ +Apache + +The following artifacts are ASL2 licensed. + +org.apache.taglibs:taglibs-standard-spec +org.apache.taglibs:taglibs-standard-impl + + +------ +MortBay + +The following artifacts are ASL2 licensed. Based on selected classes from +following Apache Tomcat jars, all ASL2 licensed. + +org.mortbay.jasper:apache-jsp + org.apache.tomcat:tomcat-jasper + org.apache.tomcat:tomcat-juli + org.apache.tomcat:tomcat-jsp-api + org.apache.tomcat:tomcat-el-api + org.apache.tomcat:tomcat-jasper-el + org.apache.tomcat:tomcat-api + org.apache.tomcat:tomcat-util-scan + org.apache.tomcat:tomcat-util + +org.mortbay.jasper:apache-el + org.apache.tomcat:tomcat-jasper-el + org.apache.tomcat:tomcat-el-api + + +------ +Mortbay + +The following artifacts are CDDL + GPLv2 with classpath exception. + +https://glassfish.dev.java.net/nonav/public/CDDL+GPL.html + +org.eclipse.jetty.toolchain:jetty-schemas + +------ +Assorted + +The UnixCrypt.java code implements the one way cryptography used by +Unix systems for simple password protection. Copyright 1996 Aki Yoshida, +modified April 2001 by Iris Van den Broeke, Daniel Deville. +Permission to use, copy, modify and distribute UnixCrypt +for non-commercial or commercial purposes and without fee is +granted provided that the copyright notice appears in all copies. + + +-------------------------------------------------------------------------------- diff --git a/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationTestBase.java b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationTestBase.java index f90d4da0379e..247211edb01f 100644 --- a/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationTestBase.java +++ b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationTestBase.java @@ -40,11 +40,13 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.assertj.core.api.Condition; import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; public class IntegrationTestBase { - private final TestContext context = TestContext.INSTANCE; + private static TestContext context; + private Catalog catalog; private Admin admin; private String connectorName; @@ -70,12 +72,16 @@ protected String testTopic() { return testTopic; } + @BeforeAll + public static void baseBeforeAll() { + context = TestContext.instance(); + } + @BeforeEach public void baseBefore() { - catalog = context.initLocalCatalog(); - producer = context.initLocalProducer(); - admin = context.initLocalAdmin(); - + this.catalog = context.initLocalCatalog(); + this.producer = context.initLocalProducer(); + this.admin = context.initLocalAdmin(); this.connectorName = "test_connector-" + UUID.randomUUID(); this.testTopic = "test-topic-" + UUID.randomUUID(); } @@ -98,7 +104,7 @@ protected void assertSnapshotProps(TableIdentifier tableIdentifier, String branc Map props = latestSnapshot(table, branch).summary(); assertThat(props) .hasKeySatisfying( - new Condition() { + new Condition<>() { @Override public boolean matches(String str) { return str.startsWith("kafka.connect.offsets."); diff --git a/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/KafkaConnectUtils.java b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/KafkaConnectUtils.java index 098ab2395b34..511923683da8 100644 --- a/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/KafkaConnectUtils.java +++ b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/KafkaConnectUtils.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.node.ArrayNode; import java.io.IOException; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.concurrent.TimeUnit; import org.apache.hc.client5.http.classic.HttpClient; @@ -66,7 +67,9 @@ public Config config(String key, Object value) { public static void startConnector(Config config) { try { HttpPost request = - new HttpPost(String.format("http://localhost:%d/connectors", TestContext.CONNECT_PORT)); + new HttpPost( + String.format( + Locale.ROOT, "http://localhost:%d/connectors", TestContext.CONNECT_PORT)); String body = TestContext.MAPPER.writeValueAsString(config); request.setHeader("Content-Type", "application/json"); request.setEntity(new StringEntity(body)); @@ -80,7 +83,10 @@ public static void ensureConnectorRunning(String name) { HttpGet request = new HttpGet( String.format( - "http://localhost:%d/connectors/%s/status", TestContext.CONNECT_PORT, name)); + Locale.ROOT, + "http://localhost:%d/connectors/%s/status", + TestContext.CONNECT_PORT, + name)); Awaitility.await() .atMost(60, TimeUnit.SECONDS) .until( @@ -106,7 +112,11 @@ public static void stopConnector(String name) { try { HttpDelete request = new HttpDelete( - String.format("http://localhost:%d/connectors/%s", TestContext.CONNECT_PORT, name)); + String.format( + Locale.ROOT, + "http://localhost:%d/connectors/%s", + TestContext.CONNECT_PORT, + name)); HTTP.execute(request, response -> null); } catch (IOException e) { throw new RuntimeException(e); diff --git a/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/TestContext.java b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/TestContext.java index 729d4bb264e5..2a1ded6cd8a1 100644 --- a/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/TestContext.java +++ b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/TestContext.java @@ -20,6 +20,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import java.io.File; +import java.time.Duration; import java.util.Map; import java.util.UUID; import org.apache.iceberg.CatalogProperties; @@ -37,7 +38,8 @@ public class TestContext { - public static final TestContext INSTANCE = new TestContext(); + private static volatile TestContext instance; + public static final ObjectMapper MAPPER = new ObjectMapper(); public static final int CONNECT_PORT = 8083; @@ -48,9 +50,17 @@ public class TestContext { private static final String AWS_SECRET_KEY = "minioadmin"; private static final String AWS_REGION = "us-east-1"; + public static synchronized TestContext instance() { + if (instance == null) { + instance = new TestContext(); + } + return instance; + } + private TestContext() { ComposeContainer container = new ComposeContainer(new File("./docker/docker-compose.yml")) + .withStartupTimeout(Duration.ofMinutes(2)) .waitingFor("connect", Wait.forHttp("/connectors")); container.start(); } diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkConfig.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkConfig.java index cf34b0bcd4c8..bf5b59a0f025 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkConfig.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkConfig.java @@ -406,6 +406,12 @@ public JsonConverter jsonConverter() { return jsonConverter; } + @VisibleForTesting + static boolean checkClassName(String className) { + return (className.matches(".*\\.ConnectDistributed.*") + || className.matches(".*\\.ConnectStandalone.*")); + } + /** * This method attempts to load the Kafka Connect worker properties, which are not exposed to * connectors. It does this by parsing the Java command used to launch the worker, extracting the @@ -422,9 +428,7 @@ private Map loadWorkerProps() { String javaCmd = System.getProperty("sun.java.command"); if (javaCmd != null && !javaCmd.isEmpty()) { List args = Splitter.on(' ').splitToList(javaCmd); - if (args.size() > 1 - && (args.get(0).endsWith(".ConnectDistributed") - || args.get(0).endsWith(".ConnectStandalone"))) { + if (args.size() > 1 && checkClassName(args.get(0))) { Properties result = new Properties(); try (InputStream in = Files.newInputStream(Paths.get(args.get(1)))) { result.load(in); diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java index 7274f77e0c85..b09540fdc9f6 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java @@ -53,6 +53,7 @@ import org.apache.iceberg.util.Tasks; import org.apache.iceberg.util.ThreadPools; import org.apache.kafka.clients.admin.MemberDescription; +import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.sink.SinkTaskContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -71,6 +72,7 @@ class Coordinator extends Channel { private final String snapshotOffsetsProp; private final ExecutorService exec; private final CommitState commitState; + private volatile boolean terminated; Coordinator( Catalog catalog, @@ -88,7 +90,7 @@ class Coordinator extends Channel { this.snapshotOffsetsProp = String.format( "kafka.connect.offsets.%s.%s", config.controlTopic(), config.connectGroupId()); - this.exec = ThreadPools.newWorkerPool("iceberg-committer", config.commitThreads()); + this.exec = ThreadPools.newFixedThreadPool("iceberg-committer", config.commitThreads()); this.commitState = new CommitState(config); } @@ -218,6 +220,10 @@ private void commitToTable( .filter(distinctByKey(deleteFile -> deleteFile.path().toString())) .collect(Collectors.toList()); + if (terminated) { + throw new ConnectException("Coordinator is terminated, commit aborted"); + } + if (dataFiles.isEmpty() && deleteFiles.isEmpty()) { LOG.info("Nothing to commit to table {}, skipping", tableIdentifier); } else { @@ -296,19 +302,18 @@ private Map lastCommittedOffsetsForTable(Table table, String bran return ImmutableMap.of(); } - @Override - void stop() { + void terminate() { + this.terminated = true; + exec.shutdownNow(); - // ensure coordinator tasks are shut down, else cause the sink worker to fail + // wait for coordinator termination, else cause the sink task to fail try { if (!exec.awaitTermination(1, TimeUnit.MINUTES)) { - throw new RuntimeException("Timed out waiting for coordinator shutdown"); + throw new ConnectException("Timed out waiting for coordinator shutdown"); } } catch (InterruptedException e) { - throw new RuntimeException("Interrupted while waiting for coordinator shutdown", e); + throw new ConnectException("Interrupted while waiting for coordinator shutdown", e); } - - super.stop(); } } diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CoordinatorThread.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CoordinatorThread.java index 6a31b17fc606..b1a34d0474d9 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CoordinatorThread.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CoordinatorThread.java @@ -25,7 +25,7 @@ class CoordinatorThread extends Thread { private static final Logger LOG = LoggerFactory.getLogger(CoordinatorThread.class); private static final String THREAD_NAME = "iceberg-coord"; - private Coordinator coordinator; + private final Coordinator coordinator; private volatile boolean terminated; CoordinatorThread(Coordinator coordinator) { @@ -39,7 +39,7 @@ public void run() { coordinator.start(); } catch (Exception e) { LOG.error("Coordinator error during start, exiting thread", e); - terminated = true; + this.terminated = true; } while (!terminated) { @@ -47,7 +47,7 @@ public void run() { coordinator.process(); } catch (Exception e) { LOG.error("Coordinator error during process, exiting thread", e); - terminated = true; + this.terminated = true; } } @@ -56,7 +56,6 @@ public void run() { } catch (Exception e) { LOG.error("Coordinator error during stop, ignoring", e); } - coordinator = null; } boolean isTerminated() { @@ -64,6 +63,7 @@ boolean isTerminated() { } void terminate() { - terminated = true; + this.terminated = true; + coordinator.terminate(); } } diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriter.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriter.java index 6df6b091510b..b5be5b3a0047 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriter.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriter.java @@ -22,6 +22,7 @@ import java.io.UncheckedIOException; import java.util.Arrays; import java.util.List; +import java.util.Locale; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.connect.IcebergSinkConfig; @@ -65,8 +66,11 @@ public void write(SinkRecord record) { } catch (Exception e) { throw new DataException( String.format( + Locale.ROOT, "An error occurred converting record, topic: %s, partition, %d, offset: %d", - record.topic(), record.kafkaPartition(), record.kafkaOffset()), + record.topic(), + record.kafkaPartition(), + record.kafkaOffset()), e); } } diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordConverter.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordConverter.java index 406a2cba4526..1a57a6444870 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordConverter.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordConverter.java @@ -37,9 +37,11 @@ import java.util.Base64; import java.util.Date; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.UUID; import java.util.stream.Collectors; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -59,8 +61,11 @@ import org.apache.iceberg.types.Types.NestedField; import org.apache.iceberg.types.Types.StructType; import org.apache.iceberg.types.Types.TimestampType; +import org.apache.iceberg.util.ByteBuffers; import org.apache.iceberg.util.DateTimeUtil; +import org.apache.iceberg.util.UUIDUtil; import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.errors.ConnectException; class RecordConverter { @@ -70,7 +75,7 @@ class RecordConverter { new DateTimeFormatterBuilder() .append(DateTimeFormatter.ISO_LOCAL_DATE_TIME) .appendOffset("+HHmm", "Z") - .toFormatter(); + .toFormatter(Locale.ROOT); private final Schema tableSchema; private final NameMapping nameMapping; @@ -128,8 +133,9 @@ private Object convertValue( case UUID: return convertUUID(value); case BINARY: - case FIXED: return convertBase64Binary(value); + case FIXED: + return ByteBuffers.toByteArray(convertBase64Binary(value)); case DATE: return convertDateValue(value); case TIME: @@ -388,13 +394,24 @@ protected String convertString(Object value) { throw new IllegalArgumentException("Cannot convert to string: " + value.getClass().getName()); } - protected UUID convertUUID(Object value) { + protected Object convertUUID(Object value) { + UUID uuid; if (value instanceof String) { - return UUID.fromString((String) value); + uuid = UUID.fromString((String) value); } else if (value instanceof UUID) { - return (UUID) value; + uuid = (UUID) value; + } else { + throw new IllegalArgumentException("Cannot convert to UUID: " + value.getClass().getName()); + } + + if (FileFormat.PARQUET + .name() + .toLowerCase(Locale.ROOT) + .equals(config.writeProps().get(TableProperties.DEFAULT_FILE_FORMAT))) { + return UUIDUtil.convert(uuid); + } else { + return uuid; } - throw new IllegalArgumentException("Cannot convert to UUID: " + value.getClass().getName()); } protected ByteBuffer convertBase64Binary(Object value) { @@ -421,7 +438,7 @@ protected LocalDate convertDateValue(Object value) { int days = (int) (((Date) value).getTime() / 1000 / 60 / 60 / 24); return DateTimeUtil.dateFromDays(days); } - throw new RuntimeException("Cannot convert date: " + value); + throw new ConnectException("Cannot convert date: " + value); } @SuppressWarnings("JavaUtilDate") @@ -437,7 +454,7 @@ protected LocalTime convertTimeValue(Object value) { long millis = ((Date) value).getTime(); return DateTimeUtil.timeFromMicros(millis * 1000); } - throw new RuntimeException("Cannot convert time: " + value); + throw new ConnectException("Cannot convert time: " + value); } protected Temporal convertTimestampValue(Object value, TimestampType type) { @@ -461,7 +478,7 @@ private OffsetDateTime convertOffsetDateTime(Object value) { } else if (value instanceof Date) { return DateTimeUtil.timestamptzFromMicros(((Date) value).getTime() * 1000); } - throw new RuntimeException( + throw new ConnectException( "Cannot convert timestamptz: " + value + ", type: " + value.getClass()); } @@ -489,7 +506,7 @@ private LocalDateTime convertLocalDateTime(Object value) { } else if (value instanceof Date) { return DateTimeUtil.timestampFromMicros(((Date) value).getTime() * 1000); } - throw new RuntimeException( + throw new ConnectException( "Cannot convert timestamp: " + value + ", type: " + value.getClass()); } diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SinkWriter.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SinkWriter.java index 35a2957f0122..f81155e13777 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SinkWriter.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SinkWriter.java @@ -23,6 +23,7 @@ import java.time.ZoneOffset; import java.util.Collection; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -119,7 +120,7 @@ private void routeRecordDynamically(SinkRecord record) { String routeValue = extractRouteValue(record.value(), routeField); if (routeValue != null) { - String tableName = routeValue.toLowerCase(); + String tableName = routeValue.toLowerCase(Locale.ROOT); writerForTable(tableName, record, true).write(record); } } diff --git a/kafka-connect/kafka-connect/src/main/resources/META-INF/services/org.apache.kafka.connect.sink.SinkConnector b/kafka-connect/kafka-connect/src/main/resources/META-INF/services/org.apache.kafka.connect.sink.SinkConnector new file mode 100644 index 000000000000..0b55dc9ff070 --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/resources/META-INF/services/org.apache.kafka.connect.sink.SinkConnector @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +org.apache.iceberg.connect.IcebergSinkConnector diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/IcebergSinkConfigTest.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/IcebergSinkConfigTest.java index e2b2f96e09ba..e774b634cbff 100644 --- a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/IcebergSinkConfigTest.java +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/IcebergSinkConfigTest.java @@ -88,4 +88,26 @@ public void testStringToList() { @Test public void testStringWithParensToList() {} + + @Test + public void testCheckClassName() { + Boolean result = + IcebergSinkConfig.checkClassName("org.apache.kafka.connect.cli.ConnectDistributed"); + assertThat(result).isTrue(); + + result = IcebergSinkConfig.checkClassName("org.apache.kafka.connect.cli.ConnectStandalone"); + assertThat(result).isTrue(); + + result = IcebergSinkConfig.checkClassName("some.other.package.ConnectDistributed"); + assertThat(result).isTrue(); + + result = IcebergSinkConfig.checkClassName("some.other.package.ConnectStandalone"); + assertThat(result).isTrue(); + + result = IcebergSinkConfig.checkClassName("some.package.ConnectDistributedWrapper"); + assertThat(result).isTrue(); + + result = IcebergSinkConfig.checkClassName("org.apache.kafka.clients.producer.KafkaProducer"); + assertThat(result).isFalse(); + } } diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CoordinatorTest.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CoordinatorTest.java index 9c0b8122ae42..f3d0cf3dcf55 100644 --- a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CoordinatorTest.java +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CoordinatorTest.java @@ -24,7 +24,6 @@ import java.time.OffsetDateTime; import java.util.List; -import java.util.Map; import java.util.UUID; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; @@ -47,14 +46,13 @@ import org.apache.iceberg.types.Types.StructType; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.connect.sink.SinkTaskContext; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; public class CoordinatorTest extends ChannelTestBase { @Test public void testCommitAppend() { - Assertions.assertEquals(0, ImmutableList.copyOf(table.snapshots().iterator()).size()); + assertThat(table.snapshots()).isEmpty(); OffsetDateTime ts = EventTestUtil.now(); UUID commitId = @@ -66,17 +64,17 @@ public void testCommitAppend() { assertCommitComplete(2, commitId, ts); List snapshots = ImmutableList.copyOf(table.snapshots()); - Assertions.assertEquals(1, snapshots.size()); + assertThat(snapshots).hasSize(1); Snapshot snapshot = snapshots.get(0); - Assertions.assertEquals(DataOperations.APPEND, snapshot.operation()); - Assertions.assertEquals(1, ImmutableList.copyOf(snapshot.addedDataFiles(table.io())).size()); - Assertions.assertEquals(0, ImmutableList.copyOf(snapshot.addedDeleteFiles(table.io())).size()); - - Map summary = snapshot.summary(); - Assertions.assertEquals(commitId.toString(), summary.get(COMMIT_ID_SNAPSHOT_PROP)); - Assertions.assertEquals("{\"0\":3}", summary.get(OFFSETS_SNAPSHOT_PROP)); - Assertions.assertEquals(ts.toString(), summary.get(VALID_THROUGH_TS_SNAPSHOT_PROP)); + assertThat(snapshot.operation()).isEqualTo(DataOperations.APPEND); + assertThat(snapshot.addedDataFiles(table.io())).hasSize(1); + assertThat(snapshot.addedDeleteFiles(table.io())).isEmpty(); + + assertThat(snapshot.summary()) + .containsEntry(COMMIT_ID_SNAPSHOT_PROP, commitId.toString()) + .containsEntry(OFFSETS_SNAPSHOT_PROP, "{\"0\":3}") + .containsEntry(VALID_THROUGH_TS_SNAPSHOT_PROP, ts.toString()); } @Test @@ -93,17 +91,17 @@ public void testCommitDelta() { assertCommitComplete(2, commitId, ts); List snapshots = ImmutableList.copyOf(table.snapshots()); - Assertions.assertEquals(1, snapshots.size()); + assertThat(snapshots).hasSize(1); Snapshot snapshot = snapshots.get(0); - Assertions.assertEquals(DataOperations.OVERWRITE, snapshot.operation()); - Assertions.assertEquals(1, ImmutableList.copyOf(snapshot.addedDataFiles(table.io())).size()); - Assertions.assertEquals(1, ImmutableList.copyOf(snapshot.addedDeleteFiles(table.io())).size()); - - Map summary = snapshot.summary(); - Assertions.assertEquals(commitId.toString(), summary.get(COMMIT_ID_SNAPSHOT_PROP)); - Assertions.assertEquals("{\"0\":3}", summary.get(OFFSETS_SNAPSHOT_PROP)); - Assertions.assertEquals(ts.toString(), summary.get(VALID_THROUGH_TS_SNAPSHOT_PROP)); + assertThat(snapshot.operation()).isEqualTo(DataOperations.OVERWRITE); + assertThat(snapshot.addedDataFiles(table.io())).hasSize(1); + assertThat(snapshot.addedDeleteFiles(table.io())).hasSize(1); + + assertThat(snapshot.summary()) + .containsEntry(COMMIT_ID_SNAPSHOT_PROP, commitId.toString()) + .containsEntry(OFFSETS_SNAPSHOT_PROP, "{\"0\":3}") + .containsEntry(VALID_THROUGH_TS_SNAPSHOT_PROP, ts.toString()); } @Test @@ -114,8 +112,7 @@ public void testCommitNoFiles() { assertThat(producer.history()).hasSize(2); assertCommitComplete(1, commitId, ts); - List snapshots = ImmutableList.copyOf(table.snapshots()); - Assertions.assertEquals(0, snapshots.size()); + assertThat(table.snapshots()).isEmpty(); } @Test @@ -136,8 +133,7 @@ public void testCommitError() { // no commit messages sent assertThat(producer.history()).hasSize(1); - List snapshots = ImmutableList.copyOf(table.snapshots()); - Assertions.assertEquals(0, snapshots.size()); + assertThat(table.snapshots()).isEmpty(); } private void assertCommitTable(int idx, UUID commitId, OffsetDateTime ts) { diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/RecordConverterTest.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/RecordConverterTest.java index b494a9da85d3..47ee76eade15 100644 --- a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/RecordConverterTest.java +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/RecordConverterTest.java @@ -37,9 +37,11 @@ import java.util.Collection; import java.util.Date; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.UUID; import java.util.function.Function; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.connect.IcebergSinkConfig; @@ -72,6 +74,7 @@ import org.apache.iceberg.types.Types.TimeType; import org.apache.iceberg.types.Types.TimestampType; import org.apache.iceberg.types.Types.UUIDType; +import org.apache.iceberg.util.UUIDUtil; import org.apache.kafka.connect.data.Decimal; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; @@ -221,6 +224,25 @@ public void testMapConvert() { assertRecordValues(record); } + @Test + public void testUUIDConversionWithParquet() { + Table table = mock(Table.class); + when(table.schema()) + .thenReturn(new org.apache.iceberg.Schema(NestedField.required(1, "uuid", UUIDType.get()))); + when(config.writeProps()) + .thenReturn( + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + FileFormat.PARQUET.name().toLowerCase(Locale.ROOT))); + + RecordConverter converter = new RecordConverter(table, config); + Map data = + ImmutableMap.builder().put("uuid", UUID_VAL.toString()).build(); + + Record record = converter.convert(data); + assertThat(record.getField("uuid")).isEqualTo(UUIDUtil.convert(UUID_VAL)); + } + @Test public void testNestedMapConvert() { Table table = mock(Table.class); @@ -859,7 +881,7 @@ public void testEvolveTypeDetectionStructNested() { assertThat(updateMap.get("st.ff").type()).isInstanceOf(DoubleType.class); } - private Map createMapData() { + public static Map createMapData() { return ImmutableMap.builder() .put("i", 1) .put("l", 2L) @@ -898,8 +920,8 @@ private Struct createStructData() { .put("s", STR_VAL) .put("b", true) .put("u", UUID_VAL.toString()) - .put("f", BYTES_VAL.array()) - .put("bi", BYTES_VAL.array()) + .put("f", BYTES_VAL) + .put("bi", BYTES_VAL) .put("li", LIST_VAL) .put("ma", MAP_VAL); } @@ -921,11 +943,11 @@ private void assertRecordValues(Record record) { assertThat(rec.getField("dec")).isEqualTo(DEC_VAL); assertThat(rec.getField("s")).isEqualTo(STR_VAL); assertThat(rec.getField("b")).isEqualTo(true); - assertThat(rec.getField("u")).isEqualTo(UUID_VAL); - assertThat(rec.getField("f")).isEqualTo(BYTES_VAL); + assertThat(rec.getField("f")).isEqualTo(BYTES_VAL.array()); assertThat(rec.getField("bi")).isEqualTo(BYTES_VAL); assertThat(rec.getField("li")).isEqualTo(LIST_VAL); assertThat(rec.getField("ma")).isEqualTo(MAP_VAL); + assertThat(rec.getField("u")).isEqualTo(UUID_VAL); } private void assertNestedRecordValues(Record record) { diff --git a/mr/src/main/java/org/apache/iceberg/mr/Catalogs.java b/mr/src/main/java/org/apache/iceberg/mr/Catalogs.java index e8a4b5da3583..2cd0e5ef4cc6 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/Catalogs.java +++ b/mr/src/main/java/org/apache/iceberg/mr/Catalogs.java @@ -225,8 +225,7 @@ public static boolean hiveCatalog(Configuration conf, Properties props) { if (catalogType != null) { return CatalogUtil.ICEBERG_CATALOG_TYPE_HIVE.equalsIgnoreCase(catalogType); } - return getCatalogProperties(conf, catalogName, catalogType).get(CatalogProperties.CATALOG_IMPL) - == null; + return getCatalogProperties(conf, catalogName).get(CatalogProperties.CATALOG_IMPL) == null; } @VisibleForTesting @@ -237,8 +236,7 @@ static Optional loadCatalog(Configuration conf, String catalogName) { } else { String name = catalogName == null ? ICEBERG_DEFAULT_CATALOG_NAME : catalogName; return Optional.of( - CatalogUtil.buildIcebergCatalog( - name, getCatalogProperties(conf, name, catalogType), conf)); + CatalogUtil.buildIcebergCatalog(name, getCatalogProperties(conf, name), conf)); } } @@ -247,11 +245,9 @@ static Optional loadCatalog(Configuration conf, String catalogName) { * * @param conf a Hadoop configuration * @param catalogName name of the catalog - * @param catalogType type of the catalog * @return complete map of catalog properties */ - private static Map getCatalogProperties( - Configuration conf, String catalogName, String catalogType) { + private static Map getCatalogProperties(Configuration conf, String catalogName) { String keyPrefix = InputFormatConfig.CATALOG_CONFIG_PREFIX + catalogName; return Streams.stream(conf.iterator()) diff --git a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java index 49f5b86aa031..637dc6d15df1 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java +++ b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java @@ -118,7 +118,7 @@ public void preCreateTable(org.apache.hadoop.hive.metastore.api.Table hmsTable) // Iceberg schema and specification generated by the code Schema schema = schema(catalogProperties, hmsTable); - PartitionSpec spec = spec(schema, catalogProperties, hmsTable); + PartitionSpec spec = spec(schema, hmsTable); // If there are partition keys specified remove them from the HMS table and add them to the // column list @@ -290,7 +290,7 @@ private Schema schema( } private static PartitionSpec spec( - Schema schema, Properties properties, org.apache.hadoop.hive.metastore.api.Table hmsTable) { + Schema schema, org.apache.hadoop.hive.metastore.api.Table hmsTable) { if (hmsTable.getParameters().get(InputFormatConfig.PARTITION_SPEC) != null) { Preconditions.checkArgument( diff --git a/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java b/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java index a222080d71db..7ea2d26891f8 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java +++ b/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java @@ -108,7 +108,7 @@ public List getSplits(JobContext context) { HiveIcebergStorageHandler.table(conf, conf.get(InputFormatConfig.TABLE_IDENTIFIER))) .orElseGet(() -> Catalogs.loadTable(conf)); final ExecutorService workerPool = - ThreadPools.newWorkerPool( + ThreadPools.newFixedThreadPool( "iceberg-plan-worker-pool", conf.getInt( SystemConfigs.WORKER_THREAD_POOL_SIZE.propertyKey(), diff --git a/mr/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormats.java b/mr/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormats.java index 86d390ca9ffe..2b93b276ad94 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormats.java +++ b/mr/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormats.java @@ -393,8 +393,10 @@ public void testWorkerPool() throws Exception { UserGroupInformation.createUserForTesting("user1", new String[] {}); UserGroupInformation user2 = UserGroupInformation.createUserForTesting("user2", new String[] {}); - final ExecutorService workerPool1 = ThreadPools.newWorkerPool("iceberg-plan-worker-pool", 1); - final ExecutorService workerPool2 = ThreadPools.newWorkerPool("iceberg-plan-worker-pool", 1); + final ExecutorService workerPool1 = + ThreadPools.newFixedThreadPool("iceberg-plan-worker-pool", 1); + final ExecutorService workerPool2 = + ThreadPools.newFixedThreadPool("iceberg-plan-worker-pool", 1); try { assertThat(getUserFromWorkerPool(user1, table, workerPool1)).isEqualTo("user1"); assertThat(getUserFromWorkerPool(user2, table, workerPool1)).isEqualTo("user1"); diff --git a/mr/src/test/java/org/apache/iceberg/mr/TestInputFormatReaderDeletes.java b/mr/src/test/java/org/apache/iceberg/mr/TestInputFormatReaderDeletes.java index a5f108969249..2cb41f11295c 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/TestInputFormatReaderDeletes.java +++ b/mr/src/test/java/org/apache/iceberg/mr/TestInputFormatReaderDeletes.java @@ -49,21 +49,18 @@ public class TestInputFormatReaderDeletes extends DeleteReadTests { private final HadoopTables tables = new HadoopTables(conf); private TestHelper helper; - // parametrized variables - @Parameter private String inputFormat; - @Parameter(index = 1) - private FileFormat fileFormat; + private String inputFormat; - @Parameters(name = "inputFormat = {0}, fileFormat = {1}") + @Parameters(name = "fileFormat = {0}, inputFormat = {1}") public static Object[][] parameters() { return new Object[][] { - {"IcebergInputFormat", FileFormat.PARQUET}, - {"IcebergInputFormat", FileFormat.AVRO}, - {"IcebergInputFormat", FileFormat.ORC}, - {"MapredIcebergInputFormat", FileFormat.PARQUET}, - {"MapredIcebergInputFormat", FileFormat.AVRO}, - {"MapredIcebergInputFormat", FileFormat.ORC}, + {FileFormat.PARQUET, "IcebergInputFormat"}, + {FileFormat.AVRO, "IcebergInputFormat"}, + {FileFormat.ORC, "IcebergInputFormat"}, + {FileFormat.PARQUET, "MapredIcebergInputFormat"}, + {FileFormat.AVRO, "MapredIcebergInputFormat"}, + {FileFormat.ORC, "MapredIcebergInputFormat"}, }; } @@ -78,9 +75,9 @@ public void writeTestDataFile() throws IOException { protected Table createTable(String name, Schema schema, PartitionSpec spec) throws IOException { Table table; - File location = temp.resolve(inputFormat).resolve(fileFormat.name()).toFile(); + File location = temp.resolve(inputFormat).resolve(format.name()).toFile(); assertThat(location.mkdirs()).isTrue(); - helper = new TestHelper(conf, tables, location.toString(), schema, spec, fileFormat, temp); + helper = new TestHelper(conf, tables, location.toString(), schema, spec, format, temp); table = helper.createTable(); TableOperations ops = ((BaseTable) table).operations(); diff --git a/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java b/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java index 8bf6bc115d13..f51aa62d02a6 100644 --- a/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java +++ b/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java @@ -272,8 +272,10 @@ public List listNamespaces(Namespace namespace) throws NoSuchNamespac org.projectnessie.model.Namespace.of(namespace.levels()); filter += String.format( + Locale.ROOT, "size(entry.keyElements) == %d && entry.encodedKey.startsWith('%s.')", - root.getElementCount() + 1, root.name()); + root.getElementCount() + 1, + root.name()); } List entries = withReference(api.getEntries()).filter(filter).stream() diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java index f0f75c842429..ca507eae575a 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java @@ -407,7 +407,7 @@ public void testDropTable() throws IOException { table.newAppend().appendFile(file1).appendFile(file2).commit(); // delete file2 - table.newDelete().deleteFile(file2.path()).commit(); + table.newDelete().deleteFile(file2).commit(); String manifestListLocation = table.currentSnapshot().manifestListLocation().replace("file:", ""); diff --git a/open-api/README.md b/open-api/README.md index 6f906ec5acef..cf13df60f7d2 100644 --- a/open-api/README.md +++ b/open-api/README.md @@ -40,3 +40,66 @@ make generate ``` The generated code is not being used in the project, but helps to see what the changes in the open-API definition are in the generated code. + +# REST Compatibility Kit (RCK) + +The REST Compatibility Kit (RCK) is a Technology Compatibility Kit (TCK) implementation for the +Iceberg REST Specification. This includes a series of tests based on the Java reference +implementation of the REST Catalog that can be executed against any REST server that implements the +spec. + +## Test Configuration + +The RCK can be configured using either environment variables or java system properties and allows +for configuring both the tests and the REST client. Environment variables prefixed by `CATALOG_` +are passed through the catalog configuring with the following mutations: + +1. The `CATALOG_` prefix is stripped from the key name +2. Single underscore (`_`) is replaced with a dot (`.`) +3. Double underscore (`__`) is replaced with a dash (`-`) +4. The key names are converted to lowercase + +A basic environment configuration would look like the following: + +```shell +CATALOG_URI=https://my_rest_server.io/ ## -> uri=https://my_rest_server.io/ +CATALOG_WAREHOUSE=test_warehouse ## -> warehouse=test_warehouse +CATALOG_IO__IMPL=org.apache.iceberg.aws.s3.S3FileIO ## -> io-impl=org.apache.iceberg.aws.s3.S3FileIO +CATALOG_CREDENTIAL=: ## -> credential=: +``` + +Java properties passed to the test must be prefixed with `rck.`, which can be used to configure some +test configurations described below and any catalog client properties. + +An example of the same configuration using java system properties would look like the following: +```shell +rck.uri=https://my_rest_server.io/ ## -> uri=https://my_rest_server.io/ +rck.warehouse=test_warehouse ## -> warehouse=test_warehouse +rck.io-impl=org.apache.iceberg.aws.s3.S3FileIO ## -> io-impl=org.apache.iceberg.aws.s3.S3FileIO +rck.credential=: ## -> credential=: +``` + +Some test behaviors are configurable depending on the catalog implementations. Not all behaviors +are strictly defined by the REST Specification. The following are currently configurable: + +| config | default | +|-------------------------------|---------| +| rck.requires-namespace-create | true | +| rck.supports-serverside-retry | true | + + +## Running Compatibility Tests + +The compatibility tests can be invoked via gradle with the following: + +Note: The default behavior is to run a local http server with a jdbc backend for testing purposes, +so `-Drck.local=false` must be set to point to an external REST server. + +```shell +./gradlew :iceberg-open-api:test --tests RESTCompatibilityKitSuite \ + -Drck.local=false \ + -Drck.requires-namespace-create=true \ + -Drck.uri=https://my_rest_server.io/ \ + -Drck.warehouse=test_warehouse \ + -Drck.credential=: +``` \ No newline at end of file diff --git a/open-api/requirements.txt b/open-api/requirements.txt index 77d4f18331ca..ba58048e02f7 100644 --- a/open-api/requirements.txt +++ b/open-api/requirements.txt @@ -16,4 +16,4 @@ # under the License. openapi-spec-validator==0.7.1 -datamodel-code-generator==0.25.9 +datamodel-code-generator==0.26.2 diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index 56212f1ac0b5..e9cce361dd88 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -54,6 +54,17 @@ class CatalogConfig(BaseModel): ..., description='Properties that should be used as default configuration; applied before client configuration.', ) + endpoints: Optional[List[str]] = Field( + None, + description='A list of endpoints that the server supports. The format of each endpoint must be " ". The HTTP verb and the resource path must be separated by a space character.', + example=[ + 'GET /v1/{prefix}/namespaces/{namespace}', + 'GET /v1/{prefix}/namespaces', + 'POST /v1/{prefix}/namespaces', + 'GET /v1/{prefix}/namespaces/{namespace}/tables/{table}', + 'GET /v1/{prefix}/namespaces/{namespace}/views/{view}', + ], + ) class UpdateNamespacePropertiesRequest(BaseModel): @@ -97,6 +108,8 @@ class ExpressionType(BaseModel): __root__: str = Field( ..., example=[ + 'true', + 'false', 'eq', 'and', 'or', @@ -118,6 +131,14 @@ class ExpressionType(BaseModel): ) +class TrueExpression(BaseModel): + type: ExpressionType + + +class FalseExpression(BaseModel): + type: ExpressionType + + class Reference(BaseModel): __root__: str = Field(..., example=['column-name']) @@ -360,6 +381,11 @@ class RemovePartitionStatisticsUpdate(BaseUpdate): snapshot_id: int = Field(..., alias='snapshot-id') +class RemovePartitionSpecsUpdate(BaseUpdate): + action: Optional[Literal['remove-partition-specs']] = None + spec_ids: List[int] = Field(..., alias='spec-ids') + + class AssertCreate(BaseModel): """ The table must not already exist; used for create transactions @@ -441,6 +467,26 @@ class AssertViewUUID(BaseModel): uuid: str +class StorageCredential(BaseModel): + prefix: str = Field( + ..., + description='Indicates a storage location prefix where the credential is relevant. Clients should choose the most specific prefix (by selecting the longest prefix) if several credentials of the same type are available.', + ) + config: Dict[str, str] + + +class LoadCredentialsResponse(BaseModel): + storage_credentials: List[StorageCredential] = Field( + ..., alias='storage-credentials' + ) + + +class PlanStatus(BaseModel): + __root__: Literal['completed', 'submitted', 'cancelled', 'failed'] = Field( + ..., description='Status of a server-side planning operation' + ) + + class RegisterTableRequest(BaseModel): name: str metadata_location: str = Field(..., alias='metadata-location') @@ -792,8 +838,8 @@ class ContentFile(BaseModel): file_path: str = Field(..., alias='file-path') file_format: FileFormat = Field(..., alias='file-format') spec_id: int = Field(..., alias='spec-id') - partition: Optional[List[PrimitiveTypeValue]] = Field( - None, + partition: List[PrimitiveTypeValue] = Field( + ..., description='A list of partition field values ordered based on the fields of the partition spec specified by the `spec-id`', example=[1, 'bar'], ) @@ -823,6 +869,20 @@ class EqualityDeleteFile(ContentFile): ) +class FieldName(BaseModel): + __root__: str = Field( + ..., + description='A full field name (including parent field names), such as those passed in APIs like Java `Schema#findField(String name)`.\nThe nested field name follows these rules - Nested struct fields are named by concatenating field names at each struct level using dot (`.`) delimiter, e.g. employer.contact_info.address.zip_code - Nested fields in a map key are named using the keyword `key`, e.g. employee_address_map.key.first_name - Nested fields in a map value are named using the keyword `value`, e.g. employee_address_map.value.zip_code - Nested fields in a list are named using the keyword `element`, e.g. employees.element.first_name', + ) + + +class PlanTask(BaseModel): + __root__: str = Field( + ..., + description='An opaque string provided by the REST server that represents a unit of work to produce file scan tasks for scan planning. This allows clients to fetch tasks across multiple requests to accommodate large result sets.', + ) + + class CreateNamespaceRequest(BaseModel): namespace: Namespace properties: Optional[Dict[str, str]] = Field( @@ -867,6 +927,29 @@ class ViewRequirement(BaseModel): __root__: AssertViewUUID = Field(..., discriminator='type') +class FailedPlanningResult(IcebergErrorResponse): + """ + Failed server-side planning result + """ + + status: Literal['failed'] + + +class AsyncPlanningResult(BaseModel): + status: Literal['submitted'] + plan_id: Optional[str] = Field( + None, alias='plan-id', description='ID used to track a planning request' + ) + + +class EmptyPlanningResult(BaseModel): + """ + Empty server-side planning result + """ + + status: Literal['cancelled'] + + class ReportMetricsRequest2(CommitReport): report_type: str = Field(..., alias='report-type') @@ -920,6 +1003,16 @@ class DataFile(ContentFile): ) +class DeleteFile(BaseModel): + __root__: Union[PositionDeleteFile, EqualityDeleteFile] = Field( + ..., discriminator='content' + ) + + +class FetchScanTasksRequest(BaseModel): + plan_task: PlanTask = Field(..., alias='plan-task') + + class Term(BaseModel): __root__: Union[Reference, TransformTerm] @@ -983,6 +1076,8 @@ class Type(BaseModel): class Expression(BaseModel): __root__: Union[ + TrueExpression, + FalseExpression, AndOrExpression, NotExpression, SetExpression, @@ -1070,6 +1165,7 @@ class TableUpdate(BaseModel): RemovePropertiesUpdate, SetStatisticsUpdate, RemoveStatisticsUpdate, + RemovePartitionSpecsUpdate, ] @@ -1108,11 +1204,16 @@ class LoadTableResult(BaseModel): The following configurations should be respected when working with tables stored in AWS S3 - `client.region`: region to configure client for making requests to AWS - - `s3.access-key-id`: id for for credentials that provide access to the data in S3 + - `s3.access-key-id`: id for credentials that provide access to the data in S3 - `s3.secret-access-key`: secret for credentials that provide access to data in S3 - `s3.session-token`: if present, this value should be used for as the session token - `s3.remote-signing-enabled`: if `true` remote signing should be performed as described in the `s3-signer-open-api.yaml` specification + ## Storage Credentials + + Credentials for ADLS / GCS / S3 / ... are provided through the `storage-credentials` field. + Clients must first check whether the respective credentials exist in the `storage-credentials` field before checking the `config` for credentials. + """ metadata_location: Optional[str] = Field( @@ -1122,6 +1223,55 @@ class LoadTableResult(BaseModel): ) metadata: TableMetadata config: Optional[Dict[str, str]] = None + storage_credentials: Optional[List[StorageCredential]] = Field( + None, alias='storage-credentials' + ) + + +class ScanTasks(BaseModel): + """ + Scan and planning tasks for server-side scan planning + + - `plan-tasks` contains opaque units of planning work + - `file-scan-tasks` contains a partial or complete list of table scan tasks + - `delete-files` contains delete files referenced by file scan tasks + + Each plan task must be passed to the fetchScanTasks endpoint to fetch the file scan tasks for the plan task. + + The list of delete files must contain all delete files referenced by the file scan tasks. + + """ + + delete_files: Optional[List[DeleteFile]] = Field( + None, + alias='delete-files', + description='Delete files referenced by file scan tasks', + ) + file_scan_tasks: Optional[List[FileScanTask]] = Field(None, alias='file-scan-tasks') + plan_tasks: Optional[List[PlanTask]] = Field(None, alias='plan-tasks') + + +class FetchPlanningResult(BaseModel): + __root__: Union[ + CompletedPlanningResult, FailedPlanningResult, EmptyPlanningResult + ] = Field( + ..., + description='Result of server-side scan planning for fetchPlanningResult', + discriminator='status', + ) + + +class PlanTableScanResult(BaseModel): + __root__: Union[ + CompletedPlanningWithIDResult, + FailedPlanningResult, + AsyncPlanningResult, + EmptyPlanningResult, + ] = Field( + ..., + description='Result of server-side scan planning for planTableScan', + discriminator='status', + ) class CommitTableRequest(BaseModel): @@ -1183,11 +1333,19 @@ class LoadViewResult(BaseModel): - `token`: Authorization bearer token to use for view requests if OAuth2 security is enabled + ## Storage Credentials + + Credentials for ADLS / GCS / S3 / ... are provided through the `storage-credentials` field. + Clients must first check whether the respective credentials exist in the `storage-credentials` field before checking the `config` for credentials. + """ metadata_location: str = Field(..., alias='metadata-location') metadata: ViewMetadata config: Optional[Dict[str, str]] = None + storage_credentials: Optional[List[StorageCredential]] = Field( + None, alias='storage-credentials' + ) class ReportMetricsRequest(BaseModel): @@ -1210,6 +1368,59 @@ class CommitTableResponse(BaseModel): metadata: TableMetadata +class PlanTableScanRequest(BaseModel): + snapshot_id: Optional[int] = Field( + None, + alias='snapshot-id', + description='Identifier for the snapshot to scan in a point-in-time scan', + ) + select: Optional[List[FieldName]] = Field( + None, description='List of selected schema fields' + ) + filter: Optional[Expression] = Field( + None, description='Expression used to filter the table data' + ) + case_sensitive: Optional[bool] = Field( + True, + alias='case-sensitive', + description='Enables case sensitive field matching for filter and select', + ) + use_snapshot_schema: Optional[bool] = Field( + False, + alias='use-snapshot-schema', + description='Whether to use the schema at the time the snapshot was written.\nWhen time travelling, the snapshot schema should be used (true). When scanning a branch, the table schema should be used (false).', + ) + start_snapshot_id: Optional[int] = Field( + None, + alias='start-snapshot-id', + description='Starting snapshot ID for an incremental scan (exclusive)', + ) + end_snapshot_id: Optional[int] = Field( + None, + alias='end-snapshot-id', + description='Ending snapshot ID for an incremental scan (inclusive).\nRequired when start-snapshot-id is specified.', + ) + stats_fields: Optional[List[FieldName]] = Field( + None, + alias='stats-fields', + description='List of fields for which the service should send column stats.', + ) + + +class FileScanTask(BaseModel): + data_file: DataFile = Field(..., alias='data-file') + delete_file_references: Optional[List[int]] = Field( + None, + alias='delete-file-references', + description='A list of indices in the delete files array (0-based)', + ) + residual_filter: Optional[Expression] = Field( + None, + alias='residual-filter', + description='An optional filter to be applied to rows in this file scan task.\nIf the residual is not present, the client must produce the residual or use the original filter.', + ) + + class Schema(StructType): schema_id: Optional[int] = Field(None, alias='schema-id') identifier_field_ids: Optional[List[int]] = Field( @@ -1217,10 +1428,31 @@ class Schema(StructType): ) +class CompletedPlanningResult(ScanTasks): + """ + Completed server-side planning result + """ + + status: Literal['completed'] + + +class FetchScanTasksResult(ScanTasks): + """ + Response schema for fetchScanTasks + """ + + class ReportMetricsRequest1(ScanReport): report_type: str = Field(..., alias='report-type') +class CompletedPlanningWithIDResult(CompletedPlanningResult): + plan_id: Optional[str] = Field( + None, alias='plan-id', description='ID used to track a planning request' + ) + status: Literal['completed'] + + StructField.update_forward_refs() ListType.update_forward_refs() MapType.update_forward_refs() @@ -1228,6 +1460,12 @@ class ReportMetricsRequest1(ScanReport): TableMetadata.update_forward_refs() ViewMetadata.update_forward_refs() AddSchemaUpdate.update_forward_refs() +ScanTasks.update_forward_refs() +FetchPlanningResult.update_forward_refs() +PlanTableScanResult.update_forward_refs() CreateTableRequest.update_forward_refs() CreateViewRequest.update_forward_refs() ReportMetricsRequest.update_forward_refs() +CompletedPlanningResult.update_forward_refs() +FetchScanTasksResult.update_forward_refs() +CompletedPlanningWithIDResult.update_forward_refs() diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index 7f0619aff4f9..9835a96e0aa3 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -100,6 +100,38 @@ paths: Common catalog configuration settings are documented at https://iceberg.apache.org/docs/latest/configuration/#catalog-properties + + + The catalog configuration also holds an optional `endpoints` field that contains information about the endpoints + supported by the server. If a server does not send the `endpoints` field, a default set of endpoints is assumed: + + - GET /v1/{prefix}/namespaces + + - POST /v1/{prefix}/namespaces + + - GET /v1/{prefix}/namespaces/{namespace} + + - DELETE /v1/{prefix}/namespaces/{namespace} + + - POST /v1/{prefix}/namespaces/{namespace}/properties + + - GET /v1/{prefix}/namespaces/{namespace}/tables + + - POST /v1/{prefix}/namespaces/{namespace}/tables + + - GET /v1/{prefix}/namespaces/{namespace}/tables/{table} + + - POST /v1/{prefix}/namespaces/{namespace}/tables/{table} + + - DELETE /v1/{prefix}/namespaces/{namespace}/tables/{table} + + - POST /v1/{prefix}/namespaces/{namespace}/register + + - POST /v1/{prefix}/namespaces/{namespace}/tables/{table}/metrics + + - POST /v1/{prefix}/tables/rename + + - POST /v1/{prefix}/transactions/commit " responses: 200: @@ -109,13 +141,20 @@ paths: schema: $ref: '#/components/schemas/CatalogConfig' example: { - "overrides": { - "warehouse": "s3://bucket/warehouse/" - }, - "defaults": { - "clients": "4" + "overrides": { + "warehouse": "s3://bucket/warehouse/" + }, + "defaults": { + "clients": "4" + }, + "endpoints": [ + "GET /v1/{prefix}/namespaces/{namespace}", + "GET /v1/{prefix}/namespaces", + "POST /v1/{prefix}/namespaces", + "GET /v1/{prefix}/namespaces/{namespace}/tables/{table}", + "GET /v1/{prefix}/namespaces/{namespace}/views/{view}" + ] } - } 400: $ref: '#/components/responses/BadRequestErrorResponse' 401: @@ -554,6 +593,261 @@ paths: 5XX: $ref: '#/components/responses/ServerErrorResponse' + /v1/{prefix}/namespaces/{namespace}/tables/{table}/plan: + parameters: + - $ref: '#/components/parameters/prefix' + - $ref: '#/components/parameters/namespace' + - $ref: '#/components/parameters/table' + post: + tags: + - Catalog API + summary: Submit a scan for planning + description: > + Submits a scan for server-side planning. + + + Point-in-time scans are planned by passing snapshot-id to identify the + table snapshot to scan. Incremental scans are planned by passing both + start-snapshot-id and end-snapshot-id. Requests that include both point + in time config properties and incremental config properties are + invalid. If the request does not include either incremental or + point-in-time config properties, scan planning should produce a + point-in-time scan of the latest snapshot in the table's main branch. + + + Responses must include a valid status listed below. A "cancelled" status is considered invalid for this endpoint. + + - When "completed" the planning operation has produced plan tasks and + file scan tasks that must be returned in the response (not fetched + later by calling fetchPlanningResult) + + - When "submitted" the response must include a plan-id used to poll + fetchPlanningResult to fetch the planning result when it is ready + + - When "failed" the response must be a valid error response + + The response for a "completed" planning operation includes two types of + tasks (file scan tasks and plan tasks) and both may be included in the + response. Tasks must not be included for any other response status. + + + Responses that include a plan-id indicate that the service is holding + state or performing work for the client. + + + - Clients should use the plan-id to fetch results from + fetchPlanningResult when the response status is "submitted" + + - Clients should inform the service if planning results are no longer + needed by calling cancelPlanning. Cancellation is not necessary after + fetchScanTasks has been used to fetch scan tasks for each plan task. + operationId: planTableScan + requestBody: + content: + application/json: + schema: + $ref: '#/components/schemas/PlanTableScanRequest' + responses: + 200: + $ref: '#/components/responses/PlanTableScanResponse' + 400: + $ref: '#/components/responses/BadRequestErrorResponse' + 401: + $ref: '#/components/responses/UnauthorizedResponse' + 403: + $ref: '#/components/responses/ForbiddenResponse' + 404: + description: + Not Found + - NoSuchTableException, the table does not exist + - NoSuchNamespaceException, the namespace does not exist + content: + application/json: + schema: + $ref: '#/components/schemas/IcebergErrorResponse' + examples: + TableDoesNotExist: + $ref: '#/components/examples/NoSuchTableError' + NamespaceDoesNotExist: + $ref: '#/components/examples/NoSuchNamespaceError' + 406: + $ref: '#/components/responses/UnsupportedOperationResponse' + 419: + $ref: '#/components/responses/AuthenticationTimeoutResponse' + 503: + $ref: '#/components/responses/ServiceUnavailableResponse' + 5XX: + $ref: '#/components/responses/ServerErrorResponse' + + /v1/{prefix}/namespaces/{namespace}/tables/{table}/plan/{plan-id}: + parameters: + - $ref: '#/components/parameters/prefix' + - $ref: '#/components/parameters/namespace' + - $ref: '#/components/parameters/table' + - $ref: '#/components/parameters/plan-id' + + get: + tags: + - Catalog API + summary: Fetches the result of scan planning for a plan-id + operationId: fetchPlanningResult + description: > + Fetches the result of scan planning for a plan-id. + + + Responses must include a valid status + + - When "completed" the planning operation has produced plan-tasks and + file-scan-tasks that must be returned in the response + + - When "submitted" the planning operation has not completed; the client + should wait to call this endpoint again to fetch a completed response + + - When "failed" the response must be a valid error response + + - When "cancelled" the plan-id is invalid and should be discarded + + + The response for a "completed" planning operation includes two types of + tasks (file scan tasks and plan tasks) and both may be included in the + response. Tasks must not be included for any other response status. + responses: + 200: + $ref: '#/components/responses/FetchPlanningResultResponse' + 400: + $ref: '#/components/responses/BadRequestErrorResponse' + 401: + $ref: '#/components/responses/UnauthorizedResponse' + 403: + $ref: '#/components/responses/ForbiddenResponse' + 404: + description: + Not Found + - NoSuchPlanIdException, the plan-id does not exist + - NoSuchTableException, the table does not exist + - NoSuchNamespaceException, the namespace does not exist + content: + application/json: + schema: + $ref: '#/components/schemas/IcebergErrorResponse' + examples: + PlanIdDoesNotExist: + $ref: '#/components/examples/NoSuchPlanIdError' + TableDoesNotExist: + $ref: '#/components/examples/NoSuchTableError' + NamespaceDoesNotExist: + $ref: '#/components/examples/NoSuchNamespaceError' + 419: + $ref: '#/components/responses/AuthenticationTimeoutResponse' + 503: + $ref: '#/components/responses/ServiceUnavailableResponse' + 5XX: + $ref: '#/components/responses/ServerErrorResponse' + + delete: + tags: + - Catalog API + summary: Cancels scan planning for a plan-id + operationId: cancelPlanning + description: > + Cancels scan planning for a plan-id. + + + This notifies the service that it can release resources held for the + scan. Clients should cancel scans that are no longer needed, either + while the plan-id returns a "submitted" status or while there are + remaining plan tasks that have not been fetched. + + + Cancellation is not necessary when + + - Scan tasks for each plan task have been fetched using fetchScanTasks + + - A plan-id has produced a "failed" or "cancelled" status from + planTableScan or fetchPlanningResult + responses: + 204: + description: Success, no content + 400: + $ref: '#/components/responses/BadRequestErrorResponse' + 401: + $ref: '#/components/responses/UnauthorizedResponse' + 403: + $ref: '#/components/responses/ForbiddenResponse' + 404: + description: + Not Found + - NoSuchTableException, the table does not exist + - NoSuchNamespaceException, the namespace does not exist + content: + application/json: + schema: + $ref: '#/components/schemas/IcebergErrorResponse' + examples: + TableDoesNotExist: + $ref: '#/components/examples/NoSuchTableError' + NamespaceDoesNotExist: + $ref: '#/components/examples/NoSuchNamespaceError' + 419: + $ref: '#/components/responses/AuthenticationTimeoutResponse' + 503: + $ref: '#/components/responses/ServiceUnavailableResponse' + 5XX: + $ref: '#/components/responses/ServerErrorResponse' + + + /v1/{prefix}/namespaces/{namespace}/tables/{table}/tasks: + parameters: + - $ref: '#/components/parameters/prefix' + - $ref: '#/components/parameters/namespace' + - $ref: '#/components/parameters/table' + + post: + tags: + - Catalog API + summary: Fetches result tasks for a plan task + operationId: fetchScanTasks + description: Fetches result tasks for a plan task. + requestBody: + content: + application/json: + schema: + $ref: '#/components/schemas/FetchScanTasksRequest' + responses: + 200: + $ref: '#/components/responses/FetchScanTasksResponse' + 400: + $ref: '#/components/responses/BadRequestErrorResponse' + 401: + $ref: '#/components/responses/UnauthorizedResponse' + 403: + $ref: '#/components/responses/ForbiddenResponse' + 404: + description: + Not Found + - NoSuchPlanTaskException, the plan-task does not exist + - NoSuchTableException, the table does not exist + - NoSuchNamespaceException, the namespace does not exist + content: + application/json: + schema: + $ref: '#/components/schemas/IcebergErrorResponse' + examples: + PlanTaskDoesNotExist: + $ref: '#/components/examples/NoSuchPlanTaskError' + TableDoesNotExist: + $ref: '#/components/examples/NoSuchTableError' + NamespaceDoesNotExist: + $ref: '#/components/examples/NoSuchNamespaceError' + 419: + $ref: '#/components/responses/AuthenticationTimeoutResponse' + 503: + $ref: '#/components/responses/ServiceUnavailableResponse' + 5XX: + $ref: '#/components/responses/ServerErrorResponse' + + + /v1/{prefix}/namespaces/{namespace}/register: parameters: - $ref: '#/components/parameters/prefix' @@ -642,7 +936,7 @@ paths: The snapshots to return in the body of the metadata. Setting the value to `all` would return the full set of snapshots currently valid for the table. Setting the value to `refs` would load all snapshots referenced by branches or tags. - + Default if no param is provided is `all`. required: false schema: @@ -741,12 +1035,12 @@ paths: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Internal Server Error", - "type": "CommitStateUnknownException", - "code": 500 + "error": { + "message": "Internal Server Error", + "type": "CommitStateUnknownException", + "code": 500 + } } - } 503: $ref: '#/components/responses/ServiceUnavailableResponse' 502: @@ -757,12 +1051,12 @@ paths: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Invalid response from the upstream server", - "type": "CommitStateUnknownException", - "code": 502 + "error": { + "message": "Invalid response from the upstream server", + "type": "CommitStateUnknownException", + "code": 502 + } } - } 504: description: A server-side gateway timeout occurred; the commit state is unknown. @@ -771,12 +1065,12 @@ paths: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Gateway timed out during commit", - "type": "CommitStateUnknownException", - "code": 504 + "error": { + "message": "Gateway timed out during commit", + "type": "CommitStateUnknownException", + "code": 504 + } } - } 5XX: description: A server-side problem that might not be addressable on the client. @@ -785,12 +1079,12 @@ paths: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Bad Gateway", - "type": "InternalServerError", - "code": 502 + "error": { + "message": "Bad Gateway", + "type": "InternalServerError", + "code": 502 + } } - } delete: tags: @@ -865,6 +1159,44 @@ paths: 5XX: $ref: '#/components/responses/ServerErrorResponse' + /v1/{prefix}/namespaces/{namespace}/tables/{table}/credentials: + parameters: + - $ref: '#/components/parameters/prefix' + - $ref: '#/components/parameters/namespace' + - $ref: '#/components/parameters/table' + + get: + tags: + - Catalog API + summary: Load vended credentials for a table from the catalog + operationId: loadCredentials + description: Load vended credentials for a table from the catalog. + responses: + 200: + $ref: '#/components/responses/LoadCredentialsResponse' + 400: + $ref: '#/components/responses/BadRequestErrorResponse' + 401: + $ref: '#/components/responses/UnauthorizedResponse' + 403: + $ref: '#/components/responses/ForbiddenResponse' + 404: + description: + Not Found - NoSuchTableException, table to load credentials for does not exist + content: + application/json: + schema: + $ref: '#/components/schemas/IcebergErrorResponse' + examples: + TableToLoadDoesNotExist: + $ref: '#/components/examples/NoSuchTableError' + 419: + $ref: '#/components/responses/AuthenticationTimeoutResponse' + 503: + $ref: '#/components/responses/ServiceUnavailableResponse' + 5XX: + $ref: '#/components/responses/ServerErrorResponse' + /v1/{prefix}/tables/rename: parameters: - $ref: '#/components/parameters/prefix' @@ -1035,12 +1367,12 @@ paths: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Internal Server Error", - "type": "CommitStateUnknownException", - "code": 500 + "error": { + "message": "Internal Server Error", + "type": "CommitStateUnknownException", + "code": 500 + } } - } 503: $ref: '#/components/responses/ServiceUnavailableResponse' 502: @@ -1051,12 +1383,12 @@ paths: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Invalid response from the upstream server", - "type": "CommitStateUnknownException", - "code": 502 + "error": { + "message": "Invalid response from the upstream server", + "type": "CommitStateUnknownException", + "code": 502 + } } - } 504: description: A server-side gateway timeout occurred; the commit state is unknown. @@ -1065,12 +1397,12 @@ paths: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Gateway timed out during commit", - "type": "CommitStateUnknownException", - "code": 504 + "error": { + "message": "Gateway timed out during commit", + "type": "CommitStateUnknownException", + "code": 504 + } } - } 5XX: description: A server-side problem that might not be addressable on the client. @@ -1079,12 +1411,12 @@ paths: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Bad Gateway", - "type": "InternalServerError", - "code": 502 + "error": { + "message": "Bad Gateway", + "type": "InternalServerError", + "code": 502 + } } - } /v1/{prefix}/namespaces/{namespace}/views: parameters: @@ -1273,12 +1605,12 @@ paths: schema: $ref: '#/components/schemas/ErrorModel' example: { - "error": { - "message": "Internal Server Error", - "type": "CommitStateUnknownException", - "code": 500 + "error": { + "message": "Internal Server Error", + "type": "CommitStateUnknownException", + "code": 500 + } } - } 503: $ref: '#/components/responses/ServiceUnavailableResponse' 502: @@ -1289,12 +1621,12 @@ paths: schema: $ref: '#/components/schemas/ErrorModel' example: { - "error": { - "message": "Invalid response from the upstream server", - "type": "CommitStateUnknownException", - "code": 502 + "error": { + "message": "Invalid response from the upstream server", + "type": "CommitStateUnknownException", + "code": 502 + } } - } 504: description: A server-side gateway timeout occurred; the commit state is unknown. @@ -1303,12 +1635,12 @@ paths: schema: $ref: '#/components/schemas/ErrorModel' example: { - "error": { - "message": "Gateway timed out during commit", - "type": "CommitStateUnknownException", - "code": 504 + "error": { + "message": "Gateway timed out during commit", + "type": "CommitStateUnknownException", + "code": 504 + } } - } 5XX: description: A server-side problem that might not be addressable on the client. @@ -1317,12 +1649,12 @@ paths: schema: $ref: '#/components/schemas/ErrorModel' example: { - "error": { - "message": "Bad Gateway", - "type": "InternalServerError", - "code": 502 + "error": { + "message": "Bad Gateway", + "type": "InternalServerError", + "code": 502 + } } - } delete: tags: @@ -1479,6 +1811,14 @@ components: type: string example: "sales" + plan-id: + name: plan-id + in: path + description: ID used to track a planning request + required: true + schema: + type: string + view: name: view in: path @@ -1583,6 +1923,19 @@ components: type: string description: Properties that should be used as default configuration; applied before client configuration. + endpoints: + type: array + items: + type: string + description: A list of endpoints that the server supports. The format of each endpoint must be " ". + The HTTP verb and the resource path must be separated by a space character. + example: [ + "GET /v1/{prefix}/namespaces/{namespace}", + "GET /v1/{prefix}/namespaces", + "POST /v1/{prefix}/namespaces", + "GET /v1/{prefix}/namespaces/{namespace}/tables/{table}", + "GET /v1/{prefix}/namespaces/{namespace}/views/{view}" + ] CreateNamespaceRequest: type: object @@ -1774,6 +2127,8 @@ components: Expression: oneOf: + - $ref: '#/components/schemas/TrueExpression' + - $ref: '#/components/schemas/FalseExpression' - $ref: '#/components/schemas/AndOrExpression' - $ref: '#/components/schemas/NotExpression' - $ref: '#/components/schemas/SetExpression' @@ -1783,6 +2138,8 @@ components: ExpressionType: type: string example: + - "true" + - "false" - "eq" - "and" - "or" @@ -1801,6 +2158,24 @@ components: - "is-nan" - "not-nan" + TrueExpression: + type: object + required: + - type + properties: + type: + $ref: '#/components/schemas/ExpressionType' + enum: ["true"] + + FalseExpression: + type: object + required: + - type + properties: + type: + $ref: '#/components/schemas/ExpressionType' + enum: ["false"] + AndOrExpression: type: object required: @@ -2268,6 +2643,7 @@ components: remove-statistics: '#/components/schemas/RemoveStatisticsUpdate' set-partition-statistics: '#/components/schemas/SetPartitionStatisticsUpdate' remove-partition-statistics: '#/components/schemas/RemovePartitionStatisticsUpdate' + remove-partition-specs: '#/components/schemas/RemovePartitionSpecsUpdate' type: object required: - action @@ -2570,6 +2946,20 @@ components: type: integer format: int64 + RemovePartitionSpecsUpdate: + allOf: + - $ref: '#/components/schemas/BaseUpdate' + required: + - spec-ids + properties: + action: + type: string + enum: [ "remove-partition-specs" ] + spec-ids: + type: array + items: + type: integer + TableUpdate: anyOf: - $ref: '#/components/schemas/AssignUUIDUpdate' @@ -2589,6 +2979,7 @@ components: - $ref: '#/components/schemas/RemovePropertiesUpdate' - $ref: '#/components/schemas/SetStatisticsUpdate' - $ref: '#/components/schemas/RemoveStatisticsUpdate' + - $ref: '#/components/schemas/RemovePartitionSpecsUpdate' ViewUpdate: anyOf: @@ -2750,6 +3141,31 @@ components: uuid: type: string + StorageCredential: + type: object + required: + - prefix + - config + properties: + prefix: + type: string + description: Indicates a storage location prefix where the credential is relevant. Clients should choose the most + specific prefix (by selecting the longest prefix) if several credentials of the same type are available. + config: + type: object + additionalProperties: + type: string + + LoadCredentialsResponse: + type: object + required: + - storage-credentials + properties: + storage-credentials: + type: array + items: + $ref: '#/components/schemas/StorageCredential' + LoadTableResult: description: | Result used when a table is successfully loaded. @@ -2772,10 +3188,15 @@ components: The following configurations should be respected when working with tables stored in AWS S3 - `client.region`: region to configure client for making requests to AWS - - `s3.access-key-id`: id for for credentials that provide access to the data in S3 + - `s3.access-key-id`: id for credentials that provide access to the data in S3 - `s3.secret-access-key`: secret for credentials that provide access to data in S3 - `s3.session-token`: if present, this value should be used for as the session token - `s3.remote-signing-enabled`: if `true` remote signing should be performed as described in the `s3-signer-open-api.yaml` specification + + ## Storage Credentials + + Credentials for ADLS / GCS / S3 / ... are provided through the `storage-credentials` field. + Clients must first check whether the respective credentials exist in the `storage-credentials` field before checking the `config` for credentials. type: object required: - metadata @@ -2789,6 +3210,144 @@ components: type: object additionalProperties: type: string + storage-credentials: + type: array + items: + $ref: '#/components/schemas/StorageCredential' + + ScanTasks: + type: object + description: > + Scan and planning tasks for server-side scan planning + + + - `plan-tasks` contains opaque units of planning work + + - `file-scan-tasks` contains a partial or complete list of table scan tasks + + - `delete-files` contains delete files referenced by file scan tasks + + + Each plan task must be passed to the fetchScanTasks endpoint to fetch + the file scan tasks for the plan task. + + + The list of delete files must contain all delete files referenced by + the file scan tasks. + properties: + delete-files: + description: Delete files referenced by file scan tasks + type: array + items: + $ref: '#/components/schemas/DeleteFile' + file-scan-tasks: + type: array + items: + $ref: '#/components/schemas/FileScanTask' + plan-tasks: + type: array + items: + $ref: '#/components/schemas/PlanTask' + + CompletedPlanningResult: + type: object + description: Completed server-side planning result + allOf: + - $ref: '#/components/schemas/ScanTasks' + - type: object + required: + - status + properties: + status: + $ref: '#/components/schemas/PlanStatus' + enum: ["completed"] + + CompletedPlanningWithIDResult: + type: object + allOf: + - $ref: '#/components/schemas/CompletedPlanningResult' + - type: object + properties: + plan-id: + description: ID used to track a planning request + type: string + + FailedPlanningResult: + type: object + description: Failed server-side planning result + allOf: + - $ref: '#/components/schemas/IcebergErrorResponse' + - type: object + required: + - status + properties: + status: + $ref: '#/components/schemas/PlanStatus' + enum: ["failed"] + + AsyncPlanningResult: + type: object + required: + - status + properties: + status: + $ref: '#/components/schemas/PlanStatus' + enum: ["submitted"] + plan-id: + description: ID used to track a planning request + type: string + + EmptyPlanningResult: + type: object + description: Empty server-side planning result + required: + - status + properties: + status: + $ref: '#/components/schemas/PlanStatus' + enum: ["submitted", "cancelled"] + + PlanStatus: + description: Status of a server-side planning operation + type: string + enum: ["completed", "submitted", "cancelled", "failed"] + + FetchPlanningResult: + type: object + description: Result of server-side scan planning for fetchPlanningResult + discriminator: + propertyName: status + mapping: + completed: '#/components/schemas/CompletedPlanningResult' + submitted: '#/components/schemas/EmptyPlanningResult' + cancelled: '#/components/schemas/EmptyPlanningResult' + failed: '#/components/schemas/FailedPlanningResult' + oneOf: + - $ref: '#/components/schemas/CompletedPlanningResult' + - $ref: '#/components/schemas/FailedPlanningResult' + - $ref: '#/components/schemas/EmptyPlanningResult' + + PlanTableScanResult: + type: object + description: Result of server-side scan planning for planTableScan + discriminator: + propertyName: status + mapping: + completed: '#/components/schemas/CompletedPlanningWithIDResult' + submitted: '#/components/schemas/AsyncPlanningResult' + cancelled: '#/components/schemas/EmptyPlanningResult' + failed: '#/components/schemas/FailedPlanningResult' + oneOf: + - $ref: '#/components/schemas/CompletedPlanningWithIDResult' + - $ref: '#/components/schemas/FailedPlanningResult' + - $ref: '#/components/schemas/AsyncPlanningResult' + - $ref: '#/components/schemas/EmptyPlanningResult' + + FetchScanTasksResult: + type: object + description: Response schema for fetchScanTasks + allOf: + - $ref: '#/components/schemas/ScanTasks' CommitTableRequest: type: object @@ -2908,6 +3467,10 @@ components: - `token`: Authorization bearer token to use for view requests if OAuth2 security is enabled + ## Storage Credentials + + Credentials for ADLS / GCS / S3 / ... are provided through the `storage-credentials` field. + Clients must first check whether the respective credentials exist in the `storage-credentials` field before checking the `config` for credentials. type: object required: - metadata-location @@ -2921,6 +3484,10 @@ components: type: object additionalProperties: type: string + storage-credentials: + type: array + items: + $ref: '#/components/schemas/StorageCredential' TokenType: type: string @@ -3059,44 +3626,45 @@ components: type: object additionalProperties: $ref: '#/components/schemas/MetricResult' - example: - "metrics": { - "total-planning-duration": { - "count": 1, - "time-unit": "nanoseconds", - "total-duration": 2644235116 - }, - "result-data-files": { - "unit": "count", - "value": 1, - }, - "result-delete-files": { - "unit": "count", - "value": 0, - }, - "total-data-manifests": { - "unit": "count", - "value": 1, - }, - "total-delete-manifests": { - "unit": "count", - "value": 0, - }, - "scanned-data-manifests": { - "unit": "count", - "value": 1, - }, - "skipped-data-manifests": { - "unit": "count", - "value": 0, - }, - "total-file-size-bytes": { - "unit": "bytes", - "value": 10, - }, - "total-delete-file-size-bytes": { - "unit": "bytes", - "value": 0, + example: { + "metrics": { + "total-planning-duration": { + "count": 1, + "time-unit": "nanoseconds", + "total-duration": 2644235116 + }, + "result-data-files": { + "unit": "count", + "value": 1, + }, + "result-delete-files": { + "unit": "count", + "value": 0, + }, + "total-data-manifests": { + "unit": "count", + "value": 1, + }, + "total-delete-manifests": { + "unit": "count", + "value": 0, + }, + "scanned-data-manifests": { + "unit": "count", + "value": 1, + }, + "skipped-data-manifests": { + "unit": "count", + "value": 0, + }, + "total-file-size-bytes": { + "unit": "bytes", + "value": 10, + }, + "total-delete-file-size-bytes": { + "unit": "bytes", + "value": 0, + } } } @@ -3584,6 +4152,7 @@ components: type: object required: - spec-id + - partition - content - file-path - file-format @@ -3603,8 +4172,8 @@ components: items: $ref: '#/components/schemas/PrimitiveTypeValue' description: - "A list of partition field values ordered based on the fields of the partition spec specified by the - `spec-id`" + A list of partition field values ordered based on the fields of + the partition spec specified by the `spec-id` example: [1, "bar"] file-size-in-bytes: type: integer @@ -3662,6 +4231,16 @@ components: - $ref: '#/components/schemas/ValueMap' description: "Map of column id to upper bound primitive type values" + DeleteFile: + discriminator: + propertyName: content + mapping: + position-deletes: '#/components/schemas/PositionDeleteFile' + equality-deletes: '#/components/schemas/EqualityDeleteFile' + oneOf: + - $ref: '#/components/schemas/PositionDeleteFile' + - $ref: '#/components/schemas/EqualityDeleteFile' + PositionDeleteFile: allOf: - $ref: '#/components/schemas/ContentFile' @@ -3687,6 +4266,106 @@ components: type: integer description: "List of equality field IDs" + PlanTableScanRequest: + type: object + properties: + snapshot-id: + description: + Identifier for the snapshot to scan in a point-in-time scan + type: integer + format: int64 + select: + description: List of selected schema fields + type: array + items: + $ref: '#/components/schemas/FieldName' + filter: + description: + Expression used to filter the table data + $ref: '#/components/schemas/Expression' + case-sensitive: + description: Enables case sensitive field matching for filter and select + type: boolean + default: true + use-snapshot-schema: + description: + Whether to use the schema at the time the snapshot was written. + + When time travelling, the snapshot schema should be used (true). + When scanning a branch, the table schema should be used (false). + type: boolean + default: false + start-snapshot-id: + description: Starting snapshot ID for an incremental scan (exclusive) + type: integer + format: int64 + end-snapshot-id: + description: + Ending snapshot ID for an incremental scan (inclusive). + + Required when start-snapshot-id is specified. + type: integer + format: int64 + stats-fields: + description: + List of fields for which the service should send column stats. + type: array + items: + $ref: '#/components/schemas/FieldName' + + FieldName: + description: + A full field name (including parent field names), such as those passed + in APIs like Java `Schema#findField(String name)`. + + The nested field name follows these rules + - Nested struct fields are named by concatenating field names at each + struct level using dot (`.`) delimiter, e.g. + employer.contact_info.address.zip_code + - Nested fields in a map key are named using the keyword `key`, e.g. + employee_address_map.key.first_name + - Nested fields in a map value are named using the keyword `value`, + e.g. employee_address_map.value.zip_code + - Nested fields in a list are named using the keyword `element`, e.g. + employees.element.first_name + type: string + + FetchScanTasksRequest: + type: object + required: + - plan-task + properties: + plan-task: + $ref: '#/components/schemas/PlanTask' + + PlanTask: + description: + An opaque string provided by the REST server that represents a + unit of work to produce file scan tasks for scan planning. This allows + clients to fetch tasks across multiple requests to accommodate large result sets. + type: string + + FileScanTask: + type: object + required: + - data-file + properties: + data-file: + $ref: '#/components/schemas/DataFile' + delete-file-references: + description: A list of indices in the delete files array (0-based) + type: array + items: + type: integer + residual-filter: + description: + An optional filter to be applied to rows in this file scan task. + + If the residual is not present, the client must produce the + residual or use the original filter. + allOf: + - $ref: '#/components/schemas/Expression' + ############################# # Reusable Response Objects # ############################# @@ -3717,12 +4396,12 @@ components: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Malformed request", - "type": "BadRequestException", - "code": 400 + "error": { + "message": "Malformed request", + "type": "BadRequestException", + "code": 400 + } } - } # Note that this is a representative example response for use as a shorthand in the spec. # The fields `message` and `type` as indicated here are not presently prescriptive. @@ -3734,12 +4413,12 @@ components: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Not authorized to make this request", - "type": "NotAuthorizedException", - "code": 401 + "error": { + "message": "Not authorized to make this request", + "type": "NotAuthorizedException", + "code": 401 + } } - } # Note that this is a representative example response for use as a shorthand in the spec. # The fields `message` and `type` as indicated here are not presently prescriptive. @@ -3750,12 +4429,12 @@ components: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Not authorized to make this request", - "type": "NotAuthorizedException", - "code": 403 + "error": { + "message": "Not authorized to make this request", + "type": "NotAuthorizedException", + "code": 403 + } } - } # Note that this is a representative example response for use as a shorthand in the spec. # The fields `message` and `type` as indicated here are not presently prescriptive. @@ -3766,12 +4445,12 @@ components: schema: $ref: '#/components/schemas/ErrorModel' example: { - "error": { - "message": "The server does not support this operation", - "type": "UnsupportedOperationException", - "code": 406 + "error": { + "message": "The server does not support this operation", + "type": "UnsupportedOperationException", + "code": 406 + } } - } IcebergErrorResponse: description: JSON wrapper for all error responses (non-2xx) @@ -3797,9 +4476,9 @@ components: schema: $ref: '#/components/schemas/CreateNamespaceResponse' example: { - "namespace": ["accounting", "tax"], - "properties": { "owner": "Ralph", "created_at": "1452120468" } - } + "namespace": ["accounting", "tax"], + "properties": { "owner": "Ralph", "created_at": "1452120468" } + } GetNamespaceResponse: description: @@ -3842,12 +4521,12 @@ components: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Credentials have timed out", - "type": "AuthenticationTimeoutException", - "code": 419 + "error": { + "message": "Credentials have timed out", + "type": "AuthenticationTimeoutException", + "code": 419 + } } - } ServiceUnavailableResponse: description: @@ -3860,12 +4539,12 @@ components: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Slow down", - "type": "SlowDownException", - "code": 503 + "error": { + "message": "Slow down", + "type": "SlowDownException", + "code": 503 + } } - } ServerErrorResponse: description: @@ -3877,12 +4556,12 @@ components: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Internal Server Error", - "type": "InternalServerError", - "code": 500 + "error": { + "message": "Internal Server Error", + "type": "InternalServerError", + "code": 500 + } } - } UpdateNamespacePropertiesResponse: description: JSON data response for a synchronous update properties request. @@ -3891,10 +4570,10 @@ components: schema: $ref: '#/components/schemas/UpdateNamespacePropertiesResponse' example: { - "updated": [ "owner" ], - "removed": [ "foo" ], - "missing": [ "bar" ] - } + "updated": [ "owner" ], + "removed": [ "foo" ], + "missing": [ "bar" ] + } CreateTableResponse: description: Table metadata result after creating a table @@ -3903,6 +4582,27 @@ components: schema: $ref: '#/components/schemas/LoadTableResult' + PlanTableScanResponse: + description: Result of submitting a table scan to plan + content: + application/json: + schema: + $ref: '#/components/schemas/PlanTableScanResult' + + FetchPlanningResultResponse: + description: Result of fetching a submitted scan planning operation + content: + application/json: + schema: + $ref: '#/components/schemas/FetchPlanningResult' + + FetchScanTasksResponse: + description: Result of retrieving additional plan tasks and file scan tasks. + content: + application/json: + schema: + $ref: '#/components/schemas/FetchScanTasksResult' + LoadTableResponse: description: Table metadata result when loading a table content: @@ -3927,6 +4627,13 @@ components: schema: $ref: '#/components/schemas/CommitTableResponse' + LoadCredentialsResponse: + description: Table credentials result when loading credentials for a table + content: + application/json: + schema: + $ref: '#/components/schemas/LoadCredentialsResponse' + ####################################### # Common examples of different values # ####################################### @@ -3935,32 +4642,32 @@ components: ListTablesEmptyExample: summary: An empty list for a namespace with no tables value: { - "identifiers": [ ] - } + "identifiers": [ ] + } ListNamespacesEmptyExample: summary: An empty list of namespaces value: { - "namespaces": [ ] - } + "namespaces": [ ] + } ListNamespacesNonEmptyExample: summary: A non-empty list of namespaces value: { - "namespaces": [ - ["accounting", "tax"], - ["accounting", "credits"] - ] - } + "namespaces": [ + ["accounting", "tax"], + ["accounting", "credits"] + ] + } ListTablesNonEmptyExample: summary: A non-empty list of table identifiers value: { - "identifiers": [ - { "namespace": ["accounting", "tax"], "name": "paid" }, - { "namespace": ["accounting", "tax"], "name": "owed" } - ] - } + "identifiers": [ + { "namespace": ["accounting", "tax"], "name": "paid" }, + { "namespace": ["accounting", "tax"], "name": "owed" } + ] + } MultipartNamespaceAsPathVariable: summary: A multi-part namespace, as represented in a path parameter @@ -3973,76 +4680,96 @@ components: NamespaceAlreadyExistsError: summary: The requested namespace already exists value: { - "error": { - "message": "The given namespace already exists", - "type": "AlreadyExistsException", - "code": 409 + "error": { + "message": "The given namespace already exists", + "type": "AlreadyExistsException", + "code": 409 + } + } + + NoSuchPlanIdError: + summary: The plan id does not exist + value: { + "error": { + "message": "The plan id does not exist", + "type": "NoSuchPlanIdException", + "code": 404 + } + } + + NoSuchPlanTaskError: + summary: The plan task does not exist + value: { + "error": { + "message": "The plan task does not exist", + "type": "NoSuchPlanTaskException", + "code": 404 + } } - } NoSuchTableError: summary: The requested table does not exist value: { - "error": { - "message": "The given table does not exist", - "type": "NoSuchTableException", - "code": 404 + "error": { + "message": "The given table does not exist", + "type": "NoSuchTableException", + "code": 404 + } } - } NoSuchViewError: summary: The requested view does not exist value: { - "error": { - "message": "The given view does not exist", - "type": "NoSuchViewException", - "code": 404 + "error": { + "message": "The given view does not exist", + "type": "NoSuchViewException", + "code": 404 + } } - } NoSuchNamespaceError: summary: The requested namespace does not exist value: { - "error": { - "message": "The given namespace does not exist", - "type": "NoSuchNamespaceException", - "code": 404 + "error": { + "message": "The given namespace does not exist", + "type": "NoSuchNamespaceException", + "code": 404 + } } - } RenameTableSameNamespace: summary: Rename a table in the same namespace value: { - "source": { "namespace": ["accounting", "tax"], "name": "paid" }, - "destination": { "namespace": ["accounting", "tax"], "name": "owed" } - } + "source": { "namespace": ["accounting", "tax"], "name": "paid" }, + "destination": { "namespace": ["accounting", "tax"], "name": "owed" } + } RenameViewSameNamespace: summary: Rename a view in the same namespace value: { - "source": { "namespace": [ "accounting", "tax" ], "name": "paid-view" }, - "destination": { "namespace": [ "accounting", "tax" ], "name": "owed-view" } - } + "source": { "namespace": [ "accounting", "tax" ], "name": "paid-view" }, + "destination": { "namespace": [ "accounting", "tax" ], "name": "owed-view" } + } TableAlreadyExistsError: summary: The requested table identifier already exists value: { - "error": { - "message": "The given table already exists", - "type": "AlreadyExistsException", - "code": 409 + "error": { + "message": "The given table already exists", + "type": "AlreadyExistsException", + "code": 409 + } } - } ViewAlreadyExistsError: summary: The requested view identifier already exists value: { - "error": { - "message": "The given view already exists", - "type": "AlreadyExistsException", - "code": 409 + "error": { + "message": "The given view already exists", + "type": "AlreadyExistsException", + "code": 409 + } } - } # This is an example response and is not meant to be prescriptive regarding the message or type. UnprocessableEntityDuplicateKey: @@ -4050,19 +4777,19 @@ components: The request body either has the same key multiple times in what should be a map with unique keys or the request body has keys in two or more fields which should be disjoint sets. value: { - "error": { - "message": "The request cannot be processed as there is a key present multiple times", - "type": "UnprocessableEntityException", - "code": 422 + "error": { + "message": "The request cannot be processed as there is a key present multiple times", + "type": "UnprocessableEntityException", + "code": 422 + } } - } UpdateAndRemoveNamespacePropertiesRequest: summary: An update namespace properties request with both properties to remove and properties to upsert. value: { - "removals": [ "foo", "bar" ], - "updates": { "owner": "Raoul" } - } + "removals": [ "foo", "bar" ], + "updates": { "owner": "Raoul" } + } securitySchemes: OAuth2: diff --git a/open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitCatalogTests.java b/open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitCatalogTests.java new file mode 100644 index 000000000000..4c4860e88a19 --- /dev/null +++ b/open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitCatalogTests.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.catalog.CatalogTests; +import org.apache.iceberg.util.PropertyUtil; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.extension.ExtendWith; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@ExtendWith(RESTServerExtension.class) +public class RESTCompatibilityKitCatalogTests extends CatalogTests { + private static final Logger LOG = LoggerFactory.getLogger(RESTCompatibilityKitCatalogTests.class); + + private static RESTCatalog restCatalog; + + @BeforeAll + static void beforeClass() throws Exception { + restCatalog = RCKUtils.initCatalogClient(); + + assertThat(restCatalog.listNamespaces()) + .withFailMessage("Namespaces list should not contain: %s", RCKUtils.TEST_NAMESPACES) + .doesNotContainAnyElementsOf(RCKUtils.TEST_NAMESPACES); + } + + @BeforeEach + void before() { + try { + RCKUtils.purgeCatalogTestEntries(restCatalog); + } catch (Exception e) { + LOG.warn("Failure during test setup", e); + } + } + + @AfterAll + static void afterClass() throws Exception { + restCatalog.close(); + } + + @Override + protected RESTCatalog catalog() { + return restCatalog; + } + + @Override + protected boolean requiresNamespaceCreate() { + return PropertyUtil.propertyAsBoolean( + restCatalog.properties(), + RESTCompatibilityKitSuite.RCK_REQUIRES_NAMESPACE_CREATE, + super.requiresNamespaceCreate()); + } + + @Override + protected boolean supportsServerSideRetry() { + return PropertyUtil.propertyAsBoolean( + restCatalog.properties(), RESTCompatibilityKitSuite.RCK_SUPPORTS_SERVERSIDE_RETRY, true); + } + + @Override + protected boolean overridesRequestedLocation() { + return PropertyUtil.propertyAsBoolean( + restCatalog.properties(), + RESTCompatibilityKitSuite.RCK_OVERRIDES_REQUESTED_LOCATION, + false); + } +} diff --git a/open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitSuite.java b/open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitSuite.java new file mode 100644 index 000000000000..a7bbe64c0650 --- /dev/null +++ b/open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitSuite.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest; + +import org.junit.platform.suite.api.SelectClasses; +import org.junit.platform.suite.api.Suite; +import org.junit.platform.suite.api.SuiteDisplayName; + +/** + * Iceberg REST Compatibility Kit + * + *

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

The tests can be configured through environment variables or system properties. By default, + * the tests will run using a local http server using a servlet implementation that leverages the + * {@link RESTCatalogAdapter}. + */ +@Suite +@SuiteDisplayName("Iceberg REST Compatibility Kit") +@SelectClasses({RESTCompatibilityKitCatalogTests.class, RESTCompatibilityKitViewCatalogTests.class}) +public class RESTCompatibilityKitSuite { + static final String RCK_REQUIRES_NAMESPACE_CREATE = "rck.requires-namespace-create"; + static final String RCK_SUPPORTS_SERVERSIDE_RETRY = "rck.supports-serverside-retry"; + static final String RCK_OVERRIDES_REQUESTED_LOCATION = "rck.overrides-requested-location"; + + protected RESTCompatibilityKitSuite() {} +} diff --git a/open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitViewCatalogTests.java b/open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitViewCatalogTests.java new file mode 100644 index 000000000000..7a18531bf6af --- /dev/null +++ b/open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitViewCatalogTests.java @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.view.ViewCatalogTests; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.extension.ExtendWith; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@ExtendWith(RESTServerExtension.class) +public class RESTCompatibilityKitViewCatalogTests extends ViewCatalogTests { + private static final Logger LOG = + LoggerFactory.getLogger(RESTCompatibilityKitViewCatalogTests.class); + private static RESTCatalog restCatalog; + + @BeforeAll + static void beforeClass() throws Exception { + restCatalog = RCKUtils.initCatalogClient(); + + assertThat(restCatalog.listNamespaces()) + .withFailMessage("Namespaces list should not contain: %s", RCKUtils.TEST_NAMESPACES) + .doesNotContainAnyElementsOf(RCKUtils.TEST_NAMESPACES); + } + + @BeforeEach + void before() { + try { + RCKUtils.purgeCatalogTestEntries(restCatalog); + } catch (Exception e) { + LOG.warn("Failure during test setup", e); + } + } + + @AfterAll + static void afterClass() throws Exception { + restCatalog.close(); + } + + @Override + protected RESTCatalog catalog() { + return restCatalog; + } + + @Override + protected Catalog tableCatalog() { + return restCatalog; + } + + @Override + protected boolean requiresNamespaceCreate() { + return PropertyUtil.propertyAsBoolean( + restCatalog.properties(), RESTCompatibilityKitSuite.RCK_REQUIRES_NAMESPACE_CREATE, true); + } + + @Override + protected boolean supportsServerSideRetry() { + return PropertyUtil.propertyAsBoolean( + restCatalog.properties(), RESTCompatibilityKitSuite.RCK_SUPPORTS_SERVERSIDE_RETRY, true); + } + + @Override + protected boolean overridesRequestedLocation() { + return PropertyUtil.propertyAsBoolean( + restCatalog.properties(), + RESTCompatibilityKitSuite.RCK_OVERRIDES_REQUESTED_LOCATION, + false); + } +} diff --git a/open-api/src/testFixtures/java/org/apache/iceberg/rest/RCKUtils.java b/open-api/src/testFixtures/java/org/apache/iceberg/rest/RCKUtils.java new file mode 100644 index 000000000000..0f1571d362e3 --- /dev/null +++ b/open-api/src/testFixtures/java/org/apache/iceberg/rest/RCKUtils.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest; + +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.PropertyUtil; + +class RCKUtils { + private static final String CATALOG_ENV_PREFIX = "CATALOG_"; + static final String RCK_LOCAL = "rck.local"; + static final String RCK_PURGE_TEST_NAMESPACES = "rck.purge-test-namespaces"; + + static final List TEST_NAMESPACES = List.of(Namespace.of("ns"), Namespace.of("newdb")); + + private RCKUtils() {} + + /** + * Utility method that allows configuring catalog properties via environment variables. + * + *

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

Examples: + * + *


+   *     CATALOG_CATALOG__IMPL=org.apache.iceberg.jdbc.JdbcCatalog -> catalog-impl=org.apache.iceberg.jdbc.JdbcCatalog
+   *     CATALOG_URI=jdbc:sqlite:memory: -> uri=jdbc:sqlite:memory:
+   *     CATALOG_WAREHOUSE=test_warehouse -> warehouse=test_warehouse
+   *     CATALOG_IO__IMPL=org.apache.iceberg.aws.s3.S3FileIO -> io-impl=org.apache.iceberg.aws.s3.S3FileIO
+   *     CATALOG_JDBC_USER=ice_user -> jdbc.user=ice_user
+   * 
+ * + * @return configuration map + */ + static Map environmentCatalogConfig() { + return System.getenv().entrySet().stream() + .filter(e -> e.getKey().startsWith(CATALOG_ENV_PREFIX)) + .collect( + Collectors.toMap( + e -> + e.getKey() + .replaceFirst(CATALOG_ENV_PREFIX, "") + .replaceAll("__", "-") + .replaceAll("_", ".") + .toLowerCase(Locale.ROOT), + Map.Entry::getValue, + (m1, m2) -> { + throw new IllegalArgumentException("Duplicate key: " + m1); + }, + HashMap::new)); + } + + static RESTCatalog initCatalogClient() { + Map catalogProperties = Maps.newHashMap(); + catalogProperties.putAll(RCKUtils.environmentCatalogConfig()); + catalogProperties.putAll(Maps.fromProperties(System.getProperties())); + + // Set defaults + catalogProperties.putIfAbsent( + CatalogProperties.URI, + String.format("http://localhost:%s/", RESTCatalogServer.REST_PORT_DEFAULT)); + catalogProperties.putIfAbsent(CatalogProperties.WAREHOUSE_LOCATION, "rck_warehouse"); + + RESTCatalog catalog = new RESTCatalog(); + catalog.setConf(new Configuration()); + catalog.initialize("rck_catalog", catalogProperties); + return catalog; + } + + static void purgeCatalogTestEntries(RESTCatalog catalog) { + if (!PropertyUtil.propertyAsBoolean(catalog.properties(), RCK_PURGE_TEST_NAMESPACES, true)) { + return; + } + + TEST_NAMESPACES.stream() + .filter(catalog::namespaceExists) + .forEach( + namespace -> { + catalog.listTables(namespace).forEach(catalog::dropTable); + catalog.listViews(namespace).forEach(catalog::dropView); + catalog.dropNamespace(namespace); + }); + } +} diff --git a/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTCatalogServer.java b/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTCatalogServer.java new file mode 100644 index 000000000000..b3d12f74e4b0 --- /dev/null +++ b/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTCatalogServer.java @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest; + +import java.io.File; +import java.io.IOException; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.jdbc.JdbcCatalog; +import org.apache.iceberg.util.PropertyUtil; +import org.eclipse.jetty.server.Server; +import org.eclipse.jetty.server.handler.gzip.GzipHandler; +import org.eclipse.jetty.servlet.ServletContextHandler; +import org.eclipse.jetty.servlet.ServletHolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class RESTCatalogServer { + private static final Logger LOG = LoggerFactory.getLogger(RESTCatalogServer.class); + + static final String REST_PORT = "rest.port"; + static final int REST_PORT_DEFAULT = 8181; + + private Server httpServer; + + RESTCatalogServer() {} + + static class CatalogContext { + private final Catalog catalog; + private final Map configuration; + + CatalogContext(Catalog catalog, Map configuration) { + this.catalog = catalog; + this.configuration = configuration; + } + + public Catalog catalog() { + return catalog; + } + + public Map configuration() { + return configuration; + } + } + + private CatalogContext initializeBackendCatalog() throws IOException { + // Translate environment variables to catalog properties + Map catalogProperties = RCKUtils.environmentCatalogConfig(); + + // Fallback to a JDBCCatalog impl if one is not set + catalogProperties.putIfAbsent(CatalogProperties.CATALOG_IMPL, JdbcCatalog.class.getName()); + catalogProperties.putIfAbsent(CatalogProperties.URI, "jdbc:sqlite::memory:"); + catalogProperties.putIfAbsent("jdbc.schema-version", "V1"); + + // Configure a default location if one is not specified + String warehouseLocation = catalogProperties.get(CatalogProperties.WAREHOUSE_LOCATION); + + if (warehouseLocation == null) { + File tmp = java.nio.file.Files.createTempDirectory("iceberg_warehouse").toFile(); + tmp.deleteOnExit(); + warehouseLocation = tmp.toPath().resolve("iceberg_data").toFile().getAbsolutePath(); + catalogProperties.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation); + + LOG.info("No warehouse location set. Defaulting to temp location: {}", warehouseLocation); + } + + LOG.info("Creating catalog with properties: {}", catalogProperties); + return new CatalogContext( + CatalogUtil.buildIcebergCatalog("rest_backend", catalogProperties, new Configuration()), + catalogProperties); + } + + public void start(boolean join) throws Exception { + CatalogContext catalogContext = initializeBackendCatalog(); + + RESTCatalogAdapter adapter = new RESTServerCatalogAdapter(catalogContext); + RESTCatalogServlet servlet = new RESTCatalogServlet(adapter); + + ServletContextHandler context = new ServletContextHandler(ServletContextHandler.NO_SESSIONS); + ServletHolder servletHolder = new ServletHolder(servlet); + context.addServlet(servletHolder, "/*"); + context.insertHandler(new GzipHandler()); + + this.httpServer = + new Server( + PropertyUtil.propertyAsInt(catalogContext.configuration, REST_PORT, REST_PORT_DEFAULT)); + httpServer.setHandler(context); + httpServer.start(); + + if (join) { + httpServer.join(); + } + } + + public void stop() throws Exception { + if (httpServer != null) { + httpServer.stop(); + } + } + + public static void main(String[] args) throws Exception { + new RESTCatalogServer().start(true); + } +} diff --git a/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTServerCatalogAdapter.java b/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTServerCatalogAdapter.java new file mode 100644 index 000000000000..612ed926f54d --- /dev/null +++ b/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTServerCatalogAdapter.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest; + +import java.util.Map; +import org.apache.iceberg.aws.s3.S3FileIOProperties; +import org.apache.iceberg.azure.AzureProperties; +import org.apache.iceberg.gcp.GCPProperties; +import org.apache.iceberg.rest.RESTCatalogServer.CatalogContext; +import org.apache.iceberg.rest.responses.LoadTableResponse; +import org.apache.iceberg.util.PropertyUtil; + +class RESTServerCatalogAdapter extends RESTCatalogAdapter { + private static final String INCLUDE_CREDENTIALS = "include-credentials"; + + private final CatalogContext catalogContext; + + RESTServerCatalogAdapter(CatalogContext catalogContext) { + super(catalogContext.catalog()); + this.catalogContext = catalogContext; + } + + @Override + public T handleRequest( + Route route, Map vars, Object body, Class responseType) { + T restResponse = super.handleRequest(route, vars, body, responseType); + + if (restResponse instanceof LoadTableResponse) { + if (PropertyUtil.propertyAsBoolean( + catalogContext.configuration(), INCLUDE_CREDENTIALS, false)) { + applyCredentials( + catalogContext.configuration(), ((LoadTableResponse) restResponse).config()); + } + } + + return restResponse; + } + + private void applyCredentials( + Map catalogConfig, Map tableConfig) { + if (catalogConfig.containsKey(S3FileIOProperties.ACCESS_KEY_ID)) { + tableConfig.put( + S3FileIOProperties.ACCESS_KEY_ID, catalogConfig.get(S3FileIOProperties.ACCESS_KEY_ID)); + } + + if (catalogConfig.containsKey(S3FileIOProperties.SECRET_ACCESS_KEY)) { + tableConfig.put( + S3FileIOProperties.SECRET_ACCESS_KEY, + catalogConfig.get(S3FileIOProperties.SECRET_ACCESS_KEY)); + } + + if (catalogConfig.containsKey(S3FileIOProperties.SESSION_TOKEN)) { + tableConfig.put( + S3FileIOProperties.SESSION_TOKEN, catalogConfig.get(S3FileIOProperties.SESSION_TOKEN)); + } + + if (catalogConfig.containsKey(GCPProperties.GCS_OAUTH2_TOKEN)) { + tableConfig.put( + GCPProperties.GCS_OAUTH2_TOKEN, catalogConfig.get(GCPProperties.GCS_OAUTH2_TOKEN)); + } + + catalogConfig.entrySet().stream() + .filter( + entry -> + entry.getKey().startsWith(AzureProperties.ADLS_SAS_TOKEN_PREFIX) + || entry.getKey().startsWith(AzureProperties.ADLS_CONNECTION_STRING_PREFIX)) + .forEach(entry -> tableConfig.put(entry.getKey(), entry.getValue())); + } +} diff --git a/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTServerExtension.java b/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTServerExtension.java new file mode 100644 index 000000000000..fc3648055694 --- /dev/null +++ b/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTServerExtension.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest; + +import org.junit.jupiter.api.extension.AfterAllCallback; +import org.junit.jupiter.api.extension.BeforeAllCallback; +import org.junit.jupiter.api.extension.ExtensionContext; + +public class RESTServerExtension implements BeforeAllCallback, AfterAllCallback { + private RESTCatalogServer localServer; + + @Override + public void beforeAll(ExtensionContext extensionContext) throws Exception { + if (Boolean.parseBoolean( + extensionContext.getConfigurationParameter(RCKUtils.RCK_LOCAL).orElse("true"))) { + this.localServer = new RESTCatalogServer(); + this.localServer.start(false); + } + } + + @Override + public void afterAll(ExtensionContext extensionContext) throws Exception { + if (localServer != null) { + localServer.stop(); + } + } +} diff --git a/orc/src/main/java/org/apache/iceberg/orc/ORC.java b/orc/src/main/java/org/apache/iceberg/orc/ORC.java index 18186fe3f20e..451c670fcd54 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/ORC.java +++ b/orc/src/main/java/org/apache/iceberg/orc/ORC.java @@ -95,7 +95,9 @@ @SuppressWarnings("checkstyle:AbbreviationAsWordInName") public class ORC { - /** @deprecated use {@link TableProperties#ORC_WRITE_BATCH_SIZE} instead */ + /** + * @deprecated use {@link TableProperties#ORC_WRITE_BATCH_SIZE} instead + */ @Deprecated private static final String VECTOR_ROW_BATCH_SIZE = "iceberg.orc.vectorbatch.size"; private ORC() {} diff --git a/orc/src/main/java/org/apache/iceberg/orc/ORCSchemaUtil.java b/orc/src/main/java/org/apache/iceberg/orc/ORCSchemaUtil.java index ba6d6eedd1d4..89a1632d5a1f 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/ORCSchemaUtil.java +++ b/orc/src/main/java/org/apache/iceberg/orc/ORCSchemaUtil.java @@ -19,6 +19,7 @@ package org.apache.iceberg.orc; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Objects; import java.util.Optional; @@ -73,6 +74,7 @@ public TypeDescription type() { * to an ORC binary type. The values for this attribute are denoted in {@code BinaryType}. */ public static final String ICEBERG_BINARY_TYPE_ATTRIBUTE = "iceberg.binary-type"; + /** * The name of the ORC {@link TypeDescription} attribute indicating the Iceberg type corresponding * to an ORC long type. The values for this attribute are denoted in {@code LongType}. @@ -326,7 +328,11 @@ private static TypeDescription buildOrcProjection( } else { if (isRequired) { throw new IllegalArgumentException( - String.format("Field %d of type %s is required and was not found.", fieldId, type)); + String.format( + Locale.ROOT, + "Field %d of type %s is required and was not found.", + fieldId, + type)); } orcType = convert(fieldId, type, false); diff --git a/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java b/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java index b8a48645efb5..842bd99f4680 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java @@ -53,7 +53,10 @@ class OrcFileAppender implements FileAppender { private final int avgRowByteSize; private final OrcRowWriter valueWriter; private boolean isClosed = false; + + @SuppressWarnings("unused") // Currently used in tests TODO remove this redundant field private final Configuration conf; + private final MetricsConfig metricsConfig; OrcFileAppender( diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/PageIterator.java b/parquet/src/main/java/org/apache/iceberg/parquet/PageIterator.java index 34383352bf68..bff13603002f 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/PageIterator.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/PageIterator.java @@ -19,6 +19,7 @@ package org.apache.iceberg.parquet; import java.io.IOException; +import java.util.Locale; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.parquet.CorruptDeltaByteArrays; import org.apache.parquet.bytes.ByteBufferInputStream; @@ -211,16 +212,26 @@ RuntimeException handleRuntimeException(RuntimeException exception) { "Read failure possibly due to " + "PARQUET-246: try setting parquet.split.files to false", new ParquetDecodingException( String.format( + Locale.ROOT, "Can't read value in column %s at value %d out of %d in current page. " + "repetition level: %d, definition level: %d", - desc, triplesRead, triplesCount, currentRL, currentDL), + desc, + triplesRead, + triplesCount, + currentRL, + currentDL), exception)); } throw new ParquetDecodingException( String.format( + Locale.ROOT, "Can't read value in column %s at value %d out of %d in current page. " + "repetition level: %d, definition level: %d", - desc, triplesRead, triplesCount, currentRL, currentDL), + desc, + triplesRead, + triplesCount, + currentRL, + currentDL), exception); } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java index 3421c0b86d3a..a3a59318320b 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java @@ -1090,7 +1090,9 @@ public ReadBuilder filter(Expression newFilter) { return this; } - /** @deprecated will be removed in 2.0.0; use {@link #createReaderFunc(Function)} instead */ + /** + * @deprecated will be removed in 2.0.0; use {@link #createReaderFunc(Function)} instead + */ @Deprecated public ReadBuilder readSupport(ReadSupport newFilterSupport) { this.readSupport = newFilterSupport; @@ -1119,7 +1121,9 @@ public ReadBuilder set(String key, String value) { return this; } - /** @deprecated will be removed in 2.0.0; use {@link #createReaderFunc(Function)} instead */ + /** + * @deprecated will be removed in 2.0.0; use {@link #createReaderFunc(Function)} instead + */ @Deprecated public ReadBuilder callInit() { this.callInit = true; diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java index 27ce35882458..a0dc54c1cdd9 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java @@ -78,7 +78,9 @@ private static Schema convertInternal( public static MessageType pruneColumns(MessageType fileSchema, Schema expectedSchema) { // column order must match the incoming type, so it doesn't matter that the ids are unordered Set selectedIds = TypeUtil.getProjectedIds(expectedSchema); - return (MessageType) ParquetTypeVisitor.visit(fileSchema, new PruneColumns(selectedIds)); + return (MessageType) + TypeWithSchemaVisitor.visit( + expectedSchema.asStruct(), fileSchema, new PruneColumns(selectedIds)); } /** diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java b/parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java index f4f50f1f3efa..d48485305e8a 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java @@ -24,6 +24,9 @@ import org.apache.iceberg.relocated.com.google.common.base.Objects; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types.ListType; +import org.apache.iceberg.types.Types.MapType; +import org.apache.iceberg.types.Types.StructType; import org.apache.parquet.schema.GroupType; import org.apache.parquet.schema.LogicalTypeAnnotation; import org.apache.parquet.schema.MessageType; @@ -31,7 +34,7 @@ import org.apache.parquet.schema.Type; import org.apache.parquet.schema.Types; -class PruneColumns extends ParquetTypeVisitor { +class PruneColumns extends TypeWithSchemaVisitor { private final Set selectedIds; PruneColumns(Set selectedIds) { @@ -40,7 +43,7 @@ class PruneColumns extends ParquetTypeVisitor { } @Override - public Type message(MessageType message, List fields) { + public Type message(StructType expected, MessageType message, List fields) { Types.MessageTypeBuilder builder = Types.buildMessage(); boolean hasChange = false; @@ -79,7 +82,7 @@ public Type message(MessageType message, List fields) { } @Override - public Type struct(GroupType struct, List fields) { + public Type struct(StructType expected, GroupType struct, List fields) { boolean hasChange = false; List filteredFields = Lists.newArrayListWithExpectedSize(fields.size()); for (int i = 0; i < fields.size(); i += 1) { @@ -106,7 +109,7 @@ public Type struct(GroupType struct, List fields) { } @Override - public Type list(GroupType list, Type element) { + public Type list(ListType expected, GroupType list, Type element) { Type repeated = list.getType(0); Type originalElement = ParquetSchemaUtil.determineListElementType(list); Integer elementId = getId(originalElement); @@ -128,7 +131,7 @@ public Type list(GroupType list, Type element) { } @Override - public Type map(GroupType map, Type key, Type value) { + public Type map(MapType expected, GroupType map, Type key, Type value) { GroupType repeated = map.getType(0).asGroupType(); Type originalKey = repeated.getType(0); Type originalValue = repeated.getType(1); @@ -150,7 +153,8 @@ public Type map(GroupType map, Type key, Type value) { } @Override - public Type primitive(PrimitiveType primitive) { + public Type primitive( + org.apache.iceberg.types.Type.PrimitiveType expected, PrimitiveType primitive) { return null; } diff --git a/pig/src/main/java/org/apache/iceberg/pig/IcebergPigInputFormat.java b/pig/src/main/java/org/apache/iceberg/pig/IcebergPigInputFormat.java index 0d5a6dd9271f..932de72ac8c0 100644 --- a/pig/src/main/java/org/apache/iceberg/pig/IcebergPigInputFormat.java +++ b/pig/src/main/java/org/apache/iceberg/pig/IcebergPigInputFormat.java @@ -56,6 +56,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * @deprecated will be removed in 1.8.0 + */ +@Deprecated public class IcebergPigInputFormat extends InputFormat { private static final Logger LOG = LoggerFactory.getLogger(IcebergPigInputFormat.class); @@ -68,6 +72,7 @@ public class IcebergPigInputFormat extends InputFormat { private List splits; IcebergPigInputFormat(Table table, String signature) { + LOG.warn("Iceberg Pig is deprecated and will be removed in Iceberg 1.8.0"); this.table = table; this.signature = signature; } diff --git a/pig/src/main/java/org/apache/iceberg/pig/IcebergStorage.java b/pig/src/main/java/org/apache/iceberg/pig/IcebergStorage.java index 88233c58a372..0ce23c39913a 100644 --- a/pig/src/main/java/org/apache/iceberg/pig/IcebergStorage.java +++ b/pig/src/main/java/org/apache/iceberg/pig/IcebergStorage.java @@ -64,6 +64,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * @deprecated will be removed in 1.8.0 + */ +@Deprecated public class IcebergStorage extends LoadFunc implements LoadMetadata, LoadPredicatePushdown, LoadPushDown { private static final Logger LOG = LoggerFactory.getLogger(IcebergStorage.class); @@ -77,6 +81,10 @@ public class IcebergStorage extends LoadFunc private IcebergRecordReader reader; + public IcebergStorage() { + LOG.warn("Iceberg Pig is deprecated and will be removed in Iceberg 1.8.0"); + } + @Override public void setLocation(String location, Job job) { LOG.info("[{}]: setLocation() -> {}", signature, location); diff --git a/pig/src/main/java/org/apache/iceberg/pig/PigParquetReader.java b/pig/src/main/java/org/apache/iceberg/pig/PigParquetReader.java index 48f15e05dd70..15ba9068caf5 100644 --- a/pig/src/main/java/org/apache/iceberg/pig/PigParquetReader.java +++ b/pig/src/main/java/org/apache/iceberg/pig/PigParquetReader.java @@ -24,6 +24,7 @@ import java.time.temporal.ChronoUnit; import java.util.LinkedHashMap; import java.util.List; +import java.util.Locale; import java.util.Map; import org.apache.iceberg.Schema; import org.apache.iceberg.parquet.ParquetSchemaUtil; @@ -59,9 +60,20 @@ import org.apache.pig.data.DataByteArray; import org.apache.pig.data.Tuple; import org.apache.pig.data.TupleFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +/** + * @deprecated will be removed in 1.8.0 + */ +@Deprecated public class PigParquetReader { - private PigParquetReader() {} + + private static final Logger LOG = LoggerFactory.getLogger(PigParquetReader.class); + + private PigParquetReader() { + LOG.warn("Iceberg Pig is deprecated and will be removed in Iceberg 1.8.0"); + } @SuppressWarnings("unchecked") public static ParquetValueReader buildReader( @@ -304,7 +316,11 @@ private static class DateReader extends PrimitiveReader { public String read(String reuse) { OffsetDateTime day = EPOCH.plusDays(column.nextInteger()); return String.format( - "%04d-%02d-%02d", day.getYear(), day.getMonth().getValue(), day.getDayOfMonth()); + Locale.ROOT, + "%04d-%02d-%02d", + day.getYear(), + day.getMonth().getValue(), + day.getDayOfMonth()); } } @@ -437,7 +453,9 @@ protected void set(Tuple tuple, int pos, Object value) { tuple.set(pos, value); } catch (ExecException e) { throw new RuntimeException( - String.format("Error setting tuple value for pos: %d, value: %s", pos, value), e); + String.format( + Locale.ROOT, "Error setting tuple value for pos: %d, value: %s", pos, value), + e); } } } diff --git a/pig/src/main/java/org/apache/iceberg/pig/SchemaUtil.java b/pig/src/main/java/org/apache/iceberg/pig/SchemaUtil.java index e02cfaf677b8..4602a5effa97 100644 --- a/pig/src/main/java/org/apache/iceberg/pig/SchemaUtil.java +++ b/pig/src/main/java/org/apache/iceberg/pig/SchemaUtil.java @@ -29,6 +29,10 @@ import org.apache.pig.data.DataType; import org.apache.pig.impl.logicalLayer.FrontendException; +/** + * @deprecated will be removed in 1.8.0 + */ +@Deprecated public class SchemaUtil { private SchemaUtil() {} diff --git a/settings.gradle b/settings.gradle index 1e6d92bf1e1f..56a68c384c5f 100644 --- a/settings.gradle +++ b/settings.gradle @@ -41,7 +41,6 @@ include 'gcp-bundle' include 'dell' include 'snowflake' include 'delta-lake' -include 'kafka-connect' include 'open-api' project(':bom').name = 'iceberg-bom' @@ -67,13 +66,13 @@ project(':gcp-bundle').name = 'iceberg-gcp-bundle' project(':dell').name = 'iceberg-dell' project(':snowflake').name = 'iceberg-snowflake' project(':delta-lake').name = 'iceberg-delta-lake' -project(':kafka-connect').name = 'iceberg-kafka-connect' project(':open-api').name = 'iceberg-open-api' if (null != System.getProperty("allModules")) { System.setProperty("flinkVersions", System.getProperty("knownFlinkVersions")) System.setProperty("sparkVersions", System.getProperty("knownSparkVersions")) System.setProperty("hiveVersions", System.getProperty("knownHiveVersions")) + System.setProperty("kafkaVersions", System.getProperty("knownKafkaVersions")) } List knownFlinkVersions = System.getProperty("knownFlinkVersions").split(",") @@ -100,6 +99,14 @@ if (!knownSparkVersions.containsAll(sparkVersions)) { throw new GradleException("Found unsupported Spark versions: " + (sparkVersions - knownSparkVersions)) } +List knownKafkaVersions = System.getProperty("knownKafkaVersions").split(",") +String kafkaVersionsString = System.getProperty("kafkaVersions") != null ? System.getProperty("kafkaVersions") : System.getProperty("defaultKafkaVersions") +List kafkaVersions = kafkaVersionsString != null && !kafkaVersionsString.isEmpty() ? kafkaVersionsString.split(",") : [] + +if (!knownKafkaVersions.containsAll(kafkaVersions)) { + throw new GradleException("Found unsupported Kafka versions: " + (kafkaVersions - knownKafkaVersions)) +} + List knownScalaVersions = System.getProperty("knownScalaVersions").split(",") String scalaVersion = System.getProperty("scalaVersion") != null ? System.getProperty("scalaVersion") : System.getProperty("defaultScalaVersion") @@ -191,14 +198,19 @@ if (hiveVersions.contains("3")) { project(':hive3-orc-bundle').name = 'iceberg-hive3-orc-bundle' } -include ":iceberg-kafka-connect:kafka-connect-events" -project(":iceberg-kafka-connect:kafka-connect-events").projectDir = file('kafka-connect/kafka-connect-events') -project(":iceberg-kafka-connect:kafka-connect-events").name = "iceberg-kafka-connect-events" +if (kafkaVersions.contains("3")) { + include 'kafka-connect' + project(':kafka-connect').name = 'iceberg-kafka-connect' -include ":iceberg-kafka-connect:kafka-connect" -project(":iceberg-kafka-connect:kafka-connect").projectDir = file('kafka-connect/kafka-connect') -project(":iceberg-kafka-connect:kafka-connect").name = "iceberg-kafka-connect" + include ":iceberg-kafka-connect:kafka-connect-events" + project(":iceberg-kafka-connect:kafka-connect-events").projectDir = file('kafka-connect/kafka-connect-events') + project(":iceberg-kafka-connect:kafka-connect-events").name = "iceberg-kafka-connect-events" -include ":iceberg-kafka-connect:kafka-connect-runtime" -project(":iceberg-kafka-connect:kafka-connect-runtime").projectDir = file('kafka-connect/kafka-connect-runtime') -project(":iceberg-kafka-connect:kafka-connect-runtime").name = "iceberg-kafka-connect-runtime" + include ":iceberg-kafka-connect:kafka-connect" + project(":iceberg-kafka-connect:kafka-connect").projectDir = file('kafka-connect/kafka-connect') + project(":iceberg-kafka-connect:kafka-connect").name = "iceberg-kafka-connect" + + include ":iceberg-kafka-connect:kafka-connect-runtime" + project(":iceberg-kafka-connect:kafka-connect-runtime").projectDir = file('kafka-connect/kafka-connect-runtime') + project(":iceberg-kafka-connect:kafka-connect-runtime").name = "iceberg-kafka-connect-runtime" +} diff --git a/site/docs/blogs.md b/site/docs/blogs.md index 00b0688e86a0..819d4f8b9e92 100644 --- a/site/docs/blogs.md +++ b/site/docs/blogs.md @@ -23,6 +23,18 @@ title: "Blogs" Here is a list of company blogs that talk about Iceberg. The blogs are ordered from most recent to oldest. + +### [Sending Data to Apache Iceberg from Apache Kafka with Apache Flink](https://www.decodable.co/blog/kafka-to-iceberg-with-flink) +**Date**: July 18th, 2024, **Company**: Decodable + +**Author**: [Robin Moffatt](https://www.linkedin.com/in/robinmoffatt) + + +### [How to get data from Apache Kafka to Apache Iceberg on S3 with Decodable](https://www.decodable.co/blog/kafka-to-iceberg-with-decodable) +**Date**: June 18th, 2024, **Company**: Decodable + +**Author**: [Robin Moffatt](https://www.linkedin.com/in/robinmoffatt) + ### [End-to-End Basic Data Engineering Tutorial (Apache Spark, Apache Iceberg, Dremio, Apache Superset, Nessie)](https://medium.com/data-engineering-with-dremio/end-to-end-basic-data-engineering-tutorial-apache-spark-apache-iceberg-dremio-apache-superset-a896ecab46f6) **Date**: April 1st, 2024, **Company**: Dremio @@ -149,6 +161,12 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Authors**: [Riza Suminto](https://www.linkedin.com/in/rizasuminto/) + +### [lakeFS ♥️ Apache Iceberg](https://lakefs.io/blog/using-lakefs-with-apache-iceberg/) +**Date**: June 26th, 2023, **Company**: LakeFS + +**Author**: [Robin Moffatt](https://www.linkedin.com/in/robinmoffatt) + ### [How Bilibili Builds OLAP Data Lakehouse with Apache Iceberg](https://medium.com/@lirui.fudan/how-bilibili-builds-olap-data-lakehouse-with-apache-iceberg-9f3408e53f9) **Date**: June 14th, 2023, **Company**: Bilibili diff --git a/site/docs/contribute.md b/site/docs/contribute.md index 60bc89f9537f..94b4679da78c 100644 --- a/site/docs/contribute.md +++ b/site/docs/contribute.md @@ -45,6 +45,18 @@ The Iceberg community prefers to receive contributions as [Github pull requests] * If a PR is related to an issue, adding `Closes #1234` in the PR description will automatically close the issue and helps keep the project clean * If a PR is posted for visibility and isn't necessarily ready for review or merging, be sure to convert the PR to a draft +### Merging Pull Requests + +Most pull requests can be merged once a single [committer](https://www.apache.org/foundation/how-it-works/#committers) other than the author is satisfied with the code in the PR (exceptions that require additional input from the community are detailed below). [Committers are trusted](https://infra.apache.org/new-committers-guide.html#the-committers-way) to act in the best [interest of the project](https://community.apache.org/projectIndependence.html#apache-projects-are-managed-independently). + +Before merging all review comments should be addressed either by making changes or agreeing the request is out of scope for the PR. For additions to public APIs committers should wait at least 24 hours before merging to ensure there is no additional feedback from members of the community. + +Requesting changes on a PR indicates a reviewer believes the PR has merit but still needs issues addressed before merging. If a reviewer believes the change should not be merged at all and there is nothing the author could do to address the reviewers concerns, the reviewer should explicitly state this on the PR. In the rare event that a PR author and reviewers cannot come to a consensus on a PR, the disagreement should be raised to the developer mailing list for further discussion. In this context, a reviewer is anyone leaving comments on the PR including contributors, committers and PMC members. + +There are several exceptions to a single committer being able to merge a PR: + +* Behavioral and functional changes to a specification must go through the [Iceberg improvement proposal](#apache-iceberg-improvement-proposals) before any code can be merged. +* Changes to files under the `format` directory and `open-api/rest-catalog*` are considered specification changes. Unless already covered under an Iceberg improvement proposal, specification changes require their own vote (e.g. bug fixes or specification clarifications). The vote follows the ASF [code modification](https://www.apache.org/foundation/voting.html#votes-on-code-modification) model and no lazy consensus modifier. Grammar, spelling and minor formatting fixes are exempted from this rule. Draft specifications (new independent specifications that are going through the Iceberg improvement process) do not require a vote but authors should provide notice on the developer mailing list about substantive changes (the final draft will be subject to a vote). ## Apache Iceberg Improvement Proposals @@ -376,6 +388,34 @@ When passing boolean arguments to existing or external methods, use inline comme dropTable(identifier, purge); ``` +#### Accessing instance variables + +Use `this` when assigning values to instance variables, making it clear when the object's state is being changed. Omit `this` when reading instance variables to keep lines shorter. + +```java + private String value; + + // BAD: unnecessary `this` during reads + public String value() { + return this.value; + } + + // GOOD: no `this` when reading instance variables + public String value() { + return value; + } + + // BAD: missing `this` in assignments + public void value(String newValue) { + value = newValue; + } + + // GOOD: use `this` in assignments + public void value(String newValue) { + this.value = newValue; + } +``` + #### Config naming 1. Use `-` to link words in one concept diff --git a/site/docs/multi-engine-support.md b/site/docs/multi-engine-support.md index a3c63276bfdb..ad67ba53cda3 100644 --- a/site/docs/multi-engine-support.md +++ b/site/docs/multi-engine-support.md @@ -93,7 +93,7 @@ Users should continuously upgrade their Flink version to stay up-to-date. | 1.17 | Deprecated | 1.3.0 | 1.6.0 | [iceberg-flink-runtime-1.17](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.17/{{ icebergVersion }}/iceberg-flink-runtime-1.17-{{ icebergVersion }}.jar) | | 1.18 | Maintained | 1.5.0 | {{ icebergVersion }} | [iceberg-flink-runtime-1.18](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.18/{{ icebergVersion }}/iceberg-flink-runtime-1.18-{{ icebergVersion }}.jar) | | 1.19 | Maintained | 1.6.0 | {{ icebergVersion }} | [iceberg-flink-runtime-1.19](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.19/{{ icebergVersion }}/iceberg-flink-runtime-1.19-{{ icebergVersion }}.jar) | -| 1.20 | Maintained | 1.7.0 | {{ icebergVersion }} | [iceberg-flink-runtime-1.20](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.20/{{ icebergVersion }}/iceberg-flink-runtime-1.20-{{ icebergVersion }}.jar) | +| 1.20 | Maintained | 1.7.0 (to be released) | - | - | diff --git a/site/mkdocs.yml b/site/mkdocs.yml index d652ec3f0830..db9bafb00f05 100644 --- a/site/mkdocs.yml +++ b/site/mkdocs.yml @@ -78,7 +78,7 @@ markdown_extensions: permalink: 🔗 extra: - icebergVersion: '1.6.0' + icebergVersion: '1.6.1' nessieVersion: '0.92.1' flinkVersion: '1.19.0' flinkVersionMajor: '1.19' diff --git a/site/nav.yml b/site/nav.yml index 91f1494a1065..c13b7cfaa6b1 100644 --- a/site/nav.yml +++ b/site/nav.yml @@ -23,6 +23,7 @@ nav: - Docs: - nightly: '!include docs/docs/nightly/mkdocs.yml' - latest: '!include docs/docs/latest/mkdocs.yml' + - 1.6.1: '!include docs/docs/1.6.1/mkdocs.yml' - 1.6.0: '!include docs/docs/1.6.0/mkdocs.yml' - 1.5.2: '!include docs/docs/1.5.2/mkdocs.yml' - 1.5.1: '!include docs/docs/1.5.1/mkdocs.yml' @@ -38,7 +39,9 @@ nav: - Vendors: vendors.md - Project: - Community: community.md - - Spec: spec.md + - Contributing: contribute.md + - REST Catalog Spec: https://editor-next.swagger.io/?url=https://raw.githubusercontent.com/apache/iceberg/main/open-api/rest-catalog-open-api.yaml + - Table Spec: spec.md - View spec: view-spec.md - Puffin spec: puffin-spec.md - AES GCM Stream spec: gcm-stream-spec.md diff --git a/site/requirements.txt b/site/requirements.txt index cb85511b7d96..5ace10851aa5 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -16,8 +16,8 @@ # under the License. mkdocs-awesome-pages-plugin==2.9.3 -mkdocs-macros-plugin==1.0.5 -mkdocs-material==9.5.31 +mkdocs-macros-plugin==1.3.7 +mkdocs-material==9.5.42 mkdocs-material-extensions==1.3.1 mkdocs-monorepo-plugin @ git+https://github.com/bitsondatadev/mkdocs-monorepo-plugin@url-fix mkdocs-redirects==1.2.1 diff --git a/snowflake/src/main/java/org/apache/iceberg/snowflake/NamespaceHelpers.java b/snowflake/src/main/java/org/apache/iceberg/snowflake/NamespaceHelpers.java index 28dacbca9817..24bd0e75aee2 100644 --- a/snowflake/src/main/java/org/apache/iceberg/snowflake/NamespaceHelpers.java +++ b/snowflake/src/main/java/org/apache/iceberg/snowflake/NamespaceHelpers.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.snowflake; +import java.util.Locale; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -47,8 +48,10 @@ public static SnowflakeIdentifier toSnowflakeIdentifier(Namespace namespace) { default: throw new IllegalArgumentException( String.format( + Locale.ROOT, "Snowflake max namespace level is %d, got namespace '%s'", - MAX_NAMESPACE_DEPTH, namespace)); + MAX_NAMESPACE_DEPTH, + namespace)); } } diff --git a/snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java b/snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java index 7f64b4ca49fd..aa324558068c 100644 --- a/snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java +++ b/snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java @@ -55,6 +55,7 @@ public class SnowflakeCatalog extends BaseMetastoreCatalog private static final String APP_IDENTIFIER = "iceberg-snowflake-catalog"; // Specifies the max length of unique id for each catalog initialized session. private static final int UNIQUE_ID_LENGTH = 20; + // Injectable factory for testing purposes. static class FileIOFactory { public FileIO newFileIO(String impl, Map properties, Object hadoopConf) { diff --git a/spark/v3.3/build.gradle b/spark/v3.3/build.gradle index c081cffeb644..87e519177470 100644 --- a/spark/v3.3/build.gradle +++ b/spark/v3.3/build.gradle @@ -187,7 +187,7 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer } project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersion}") { - apply plugin: 'io.github.goooler.shadow' + apply plugin: 'com.gradleup.shadow' tasks.jar.dependsOn tasks.shadowJar diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java index 44ee2ebdb646..b24ebfcf7977 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java @@ -939,6 +939,28 @@ public void testPartitionedImportFromEmptyPartitionDoesNotThrow() { sql("SELECT * FROM %s ORDER BY id", tableName)); } + @Test + public void testAddFilesWithParallelism() { + createUnpartitionedHiveTable(); + + String createIceberg = + "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg"; + + sql(createIceberg, tableName); + + List result = + sql( + "CALL %s.system.add_files(table => '%s', source_table => '%s', parallelism => 2)", + catalogName, tableName, sourceTableName); + + assertEquals("Procedure output must match", ImmutableList.of(row(2L, 1L)), result); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT * FROM %s ORDER BY id", sourceTableName), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + private static final List EMPTY_QUERY_RESULT = Lists.newArrayList(); private static final StructField[] STRUCT = { diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java index 281bfbf7ad1b..c65be60cc1a7 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java @@ -20,6 +20,8 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.sql.Timestamp; +import java.time.Instant; import java.time.LocalDateTime; import java.util.List; import java.util.Map; @@ -252,6 +254,39 @@ public void testRollbackToTimestampWithoutExplicitCatalog() { sql("SELECT * FROM %s ORDER BY id", tableName)); } + @Test + public void testRollbackToTimestampBeforeOrEqualToOldestSnapshot() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot firstSnapshot = table.currentSnapshot(); + Timestamp beforeFirstSnapshot = + Timestamp.from(Instant.ofEpochMilli(firstSnapshot.timestampMillis() - 1)); + Timestamp exactFirstSnapshot = + Timestamp.from(Instant.ofEpochMilli(firstSnapshot.timestampMillis())); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rollback_to_timestamp(timestamp => TIMESTAMP '%s', table => '%s')", + catalogName, beforeFirstSnapshot, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot roll back, no valid snapshot older than: %s", + beforeFirstSnapshot.toInstant().toEpochMilli()); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rollback_to_timestamp(timestamp => TIMESTAMP '%s', table => '%s')", + catalogName, exactFirstSnapshot, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot roll back, no valid snapshot older than: %s", + exactFirstSnapshot.toInstant().toEpochMilli()); + } + @Test public void testInvalidRollbackToTimestampCases() { String timestamp = "TIMESTAMP '2007-12-03T10:15:30'"; diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java index 5b7b9097755b..06180cc9cf49 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java @@ -35,6 +35,7 @@ public class TestSnapshotTableProcedure extends SparkExtensionsTestBase { private static final String SOURCE_NAME = "spark_catalog.default.source"; + // Currently we can only Snapshot only out of the Spark Session Catalog public TestSnapshotTableProcedure( diff --git a/spark/v3.3/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java b/spark/v3.3/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java index d6fab897d62d..51e3721aea4f 100644 --- a/spark/v3.3/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java +++ b/spark/v3.3/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java @@ -101,7 +101,7 @@ public void testAlterTable() throws NoSuchTableException { sql( "CREATE TABLE %s (category int, id bigint, data string, ts timestamp) USING iceberg", tableName); - Table table = getTable(); + Table table; // Add examples sql("ALTER TABLE %s ADD PARTITION FIELD bucket(16, id)", tableName); sql("ALTER TABLE %s ADD PARTITION FIELD truncate(data, 4)", tableName); diff --git a/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java b/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java index 5a7df7283728..b6ade2bff3e1 100644 --- a/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java +++ b/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java @@ -22,6 +22,7 @@ import java.sql.Timestamp; import java.util.List; +import java.util.Locale; import java.util.UUID; import java.util.concurrent.TimeUnit; import org.apache.iceberg.AppendFiles; @@ -124,7 +125,7 @@ private void appendData() { for (int i = 0; i < NUM_SNAPSHOTS; i++) { AppendFiles appendFiles = table().newFastAppend(); for (int j = 0; j < NUM_FILES; j++) { - String path = String.format("%s/path/to/data-%d-%d.parquet", location, i, j); + String path = String.format(Locale.ROOT, "%s/path/to/data-%d-%d.parquet", location, i, j); validAndOrphanPaths.add(path); DataFile dataFile = DataFiles.builder(partitionSpec) diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java index af1e99df71d3..dfd0b58ffbee 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java @@ -28,6 +28,7 @@ import java.util.List; import java.util.Map; import java.util.UUID; +import java.util.concurrent.ExecutorService; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -291,7 +292,7 @@ public static List listPartition( PartitionSpec spec, SerializableConfiguration conf, MetricsConfig metricsConfig) { - return listPartition(partition, spec, conf, metricsConfig, null); + return listPartition(partition, spec, conf, metricsConfig, null, 1); } /** @@ -314,7 +315,8 @@ public static List listPartition( PartitionSpec spec, SerializableConfiguration conf, MetricsConfig metricsConfig, - NameMapping mapping) { + NameMapping mapping, + int parallelism) { return TableMigrationUtil.listPartition( partition.values, partition.uri, @@ -322,7 +324,26 @@ public static List listPartition( spec, conf.get(), metricsConfig, - mapping); + mapping, + parallelism); + } + + private static List listPartition( + SparkPartition partition, + PartitionSpec spec, + SerializableConfiguration conf, + MetricsConfig metricsConfig, + NameMapping mapping, + ExecutorService service) { + return TableMigrationUtil.listPartition( + partition.values, + partition.uri, + partition.format, + spec, + conf.get(), + metricsConfig, + mapping, + service); } private static SparkPartition toSparkPartition( @@ -419,6 +440,114 @@ public static void importSparkTable( String stagingDir, Map partitionFilter, boolean checkDuplicateFiles) { + importSparkTable( + spark, sourceTableIdent, targetTable, stagingDir, partitionFilter, checkDuplicateFiles, 1); + } + + /** + * Import files from an existing Spark table to an Iceberg table. + * + *

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

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

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

The import uses the Spark session to get table metadata. It assumes no operation is going on + * the original and target table and thus is not thread-safe. + * + * @param spark a Spark session + * @param sourceTableIdent an identifier of the source Spark table + * @param targetTable an Iceberg table where to import the data + * @param stagingDir a staging directory to store temporary manifest files + * @param partitionFilter only import partitions whose values match those in the map, can be + * partially defined + * @param checkDuplicateFiles if true, throw exception if import results in a duplicate data file + * @param service executor service to use for file reading + */ + public static void importSparkTable( + SparkSession spark, + TableIdentifier sourceTableIdent, + Table targetTable, + String stagingDir, + Map partitionFilter, + boolean checkDuplicateFiles, + ExecutorService service) { SessionCatalog catalog = spark.sessionState().catalog(); String db = @@ -439,7 +568,7 @@ public static void importSparkTable( if (Objects.equal(spec, PartitionSpec.unpartitioned())) { importUnpartitionedSparkTable( - spark, sourceTableIdentWithDB, targetTable, checkDuplicateFiles); + spark, sourceTableIdentWithDB, targetTable, checkDuplicateFiles, service); } else { List sourceTablePartitions = getPartitions(spark, sourceTableIdent, partitionFilter); @@ -447,7 +576,13 @@ public static void importSparkTable( targetTable.newAppend().commit(); } else { importSparkPartitions( - spark, sourceTablePartitions, targetTable, spec, stagingDir, checkDuplicateFiles); + spark, + sourceTablePartitions, + targetTable, + spec, + stagingDir, + checkDuplicateFiles, + service); } } } catch (AnalysisException e) { @@ -480,7 +615,8 @@ public static void importSparkTable( targetTable, stagingDir, Collections.emptyMap(), - checkDuplicateFiles); + checkDuplicateFiles, + 1); } /** @@ -497,14 +633,15 @@ public static void importSparkTable( public static void importSparkTable( SparkSession spark, TableIdentifier sourceTableIdent, Table targetTable, String stagingDir) { importSparkTable( - spark, sourceTableIdent, targetTable, stagingDir, Collections.emptyMap(), false); + spark, sourceTableIdent, targetTable, stagingDir, Collections.emptyMap(), false, 1); } private static void importUnpartitionedSparkTable( SparkSession spark, TableIdentifier sourceTableIdent, Table targetTable, - boolean checkDuplicateFiles) { + boolean checkDuplicateFiles, + ExecutorService service) { try { CatalogTable sourceTable = spark.sessionState().catalog().getTableMetadata(sourceTableIdent); Option format = @@ -529,7 +666,8 @@ private static void importUnpartitionedSparkTable( spec, conf, metricsConfig, - nameMapping); + nameMapping, + service); if (checkDuplicateFiles) { Dataset importedFiles = @@ -577,9 +715,60 @@ public static void importSparkPartitions( PartitionSpec spec, String stagingDir, boolean checkDuplicateFiles) { + importSparkPartitions(spark, partitions, targetTable, spec, stagingDir, checkDuplicateFiles, 1); + } + + /** + * Import files from given partitions to an Iceberg table. + * + * @param spark a Spark session + * @param partitions partitions to import + * @param targetTable an Iceberg table where to import the data + * @param spec a partition spec + * @param stagingDir a staging directory to store temporary manifest files + * @param checkDuplicateFiles if true, throw exception if import results in a duplicate data file + * @param parallelism number of threads to use for file reading + */ + public static void importSparkPartitions( + SparkSession spark, + List partitions, + Table targetTable, + PartitionSpec spec, + String stagingDir, + boolean checkDuplicateFiles, + int parallelism) { + importSparkPartitions( + spark, + partitions, + targetTable, + spec, + stagingDir, + checkDuplicateFiles, + TableMigrationUtil.migrationService(parallelism)); + } + + /** + * Import files from given partitions to an Iceberg table. + * + * @param spark a Spark session + * @param partitions partitions to import + * @param targetTable an Iceberg table where to import the data + * @param spec a partition spec + * @param stagingDir a staging directory to store temporary manifest files + * @param checkDuplicateFiles if true, throw exception if import results in a duplicate data file + * @param service executor service to use for file reading + */ + public static void importSparkPartitions( + SparkSession spark, + List partitions, + Table targetTable, + PartitionSpec spec, + String stagingDir, + boolean checkDuplicateFiles, + ExecutorService service) { Configuration conf = spark.sessionState().newHadoopConf(); SerializableConfiguration serializableConf = new SerializableConfiguration(conf); - int parallelism = + int listingParallelism = Math.min( partitions.size(), spark.sessionState().conf().parallelPartitionDiscoveryParallelism()); int numShufflePartitions = spark.sessionState().conf().numShufflePartitions(); @@ -589,7 +778,7 @@ public static void importSparkPartitions( nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null; JavaSparkContext sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); - JavaRDD partitionRDD = sparkContext.parallelize(partitions, parallelism); + JavaRDD partitionRDD = sparkContext.parallelize(partitions, listingParallelism); Dataset partitionDS = spark.createDataset(partitionRDD.rdd(), Encoders.javaSerialization(SparkPartition.class)); @@ -599,7 +788,12 @@ public static void importSparkPartitions( (FlatMapFunction) sparkPartition -> listPartition( - sparkPartition, spec, serializableConf, metricsConfig, nameMapping) + sparkPartition, + spec, + serializableConf, + metricsConfig, + nameMapping, + service) .iterator(), Encoders.javaSerialization(DataFile.class)); @@ -672,7 +866,7 @@ public static void importSparkPartitions( Table targetTable, PartitionSpec spec, String stagingDir) { - importSparkPartitions(spark, partitions, targetTable, spec, stagingDir, false); + importSparkPartitions(spark, partitions, targetTable, spec, stagingDir, false, 1); } public static List filterPartitions( diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateTableSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateTableSparkAction.java index fe8acf0157d3..0eb2a99f2f49 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateTableSparkAction.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateTableSparkAction.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.actions; import java.util.Map; +import java.util.concurrent.ExecutorService; import org.apache.iceberg.Snapshot; import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.Table; @@ -59,6 +60,7 @@ public class MigrateTableSparkAction extends BaseTableCreationSparkAction partitionFilter, - boolean checkDuplicateFiles) { + boolean checkDuplicateFiles, + int parallelism) { return modifyIcebergTable( destIdent, table -> { @@ -153,9 +164,16 @@ private InternalRow[] importToIceberg( Path sourcePath = new Path(sourceIdent.name()); String format = sourceIdent.namespace()[0]; importFileTable( - table, sourcePath, format, partitionFilter, checkDuplicateFiles, table.spec()); + table, + sourcePath, + format, + partitionFilter, + checkDuplicateFiles, + table.spec(), + parallelism); } else { - importCatalogTable(table, sourceIdent, partitionFilter, checkDuplicateFiles); + importCatalogTable( + table, sourceIdent, partitionFilter, checkDuplicateFiles, parallelism); } Snapshot snapshot = table.currentSnapshot(); @@ -178,7 +196,8 @@ private void importFileTable( String format, Map partitionFilter, boolean checkDuplicateFiles, - PartitionSpec spec) { + PartitionSpec spec, + int parallelism) { // List Partitions via Spark InMemory file search interface List partitions = Spark3Util.getPartitions(spark(), tableLocation, format, partitionFilter, spec); @@ -193,11 +212,11 @@ private void importFileTable( // Build a Global Partition for the source SparkPartition partition = new SparkPartition(Collections.emptyMap(), tableLocation.toString(), format); - importPartitions(table, ImmutableList.of(partition), checkDuplicateFiles); + importPartitions(table, ImmutableList.of(partition), checkDuplicateFiles, parallelism); } else { Preconditions.checkArgument( !partitions.isEmpty(), "Cannot find any matching partitions in table %s", table.name()); - importPartitions(table, partitions, checkDuplicateFiles); + importPartitions(table, partitions, checkDuplicateFiles, parallelism); } } @@ -205,7 +224,8 @@ private void importCatalogTable( Table table, Identifier sourceIdent, Map partitionFilter, - boolean checkDuplicateFiles) { + boolean checkDuplicateFiles, + int parallelism) { String stagingLocation = getMetadataLocation(table); TableIdentifier sourceTableIdentifier = Spark3Util.toV1TableIdentifier(sourceIdent); SparkTableUtil.importSparkTable( @@ -214,14 +234,24 @@ private void importCatalogTable( table, stagingLocation, partitionFilter, - checkDuplicateFiles); + checkDuplicateFiles, + parallelism); } private void importPartitions( - Table table, List partitions, boolean checkDuplicateFiles) { + Table table, + List partitions, + boolean checkDuplicateFiles, + int parallelism) { String stagingLocation = getMetadataLocation(table); SparkTableUtil.importSparkPartitions( - spark(), partitions, table, table.spec(), stagingLocation, checkDuplicateFiles); + spark(), + partitions, + table, + table.spec(), + stagingLocation, + checkDuplicateFiles, + parallelism); } private String getMetadataLocation(Table table) { diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/MigrateTableProcedure.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/MigrateTableProcedure.java index aaa6d2cb238d..69e4ef20ea50 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/MigrateTableProcedure.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/MigrateTableProcedure.java @@ -39,7 +39,8 @@ class MigrateTableProcedure extends BaseProcedure { new ProcedureParameter[] { ProcedureParameter.required("table", DataTypes.StringType), ProcedureParameter.optional("properties", STRING_MAP), - ProcedureParameter.optional("drop_backup", DataTypes.BooleanType) + ProcedureParameter.optional("drop_backup", DataTypes.BooleanType), + ProcedureParameter.optional("parallelism", DataTypes.IntegerType) }; private static final StructType OUTPUT_TYPE = @@ -95,13 +96,19 @@ public InternalRow[] call(InternalRow args) { MigrateTableSparkAction migrateTableSparkAction = SparkActions.get().migrateTable(tableName).tableProperties(properties); - MigrateTable.Result result; if (dropBackup) { - result = migrateTableSparkAction.dropBackup().execute(); - } else { - result = migrateTableSparkAction.execute(); + migrateTableSparkAction = migrateTableSparkAction.dropBackup(); } + if (!args.isNullAt(3)) { + int parallelism = args.getInt(3); + Preconditions.checkArgument(parallelism > 0, "Parallelism should be larger than 0"); + migrateTableSparkAction = + migrateTableSparkAction.executeWith(executorService(parallelism, "table-migration")); + } + + MigrateTable.Result result = migrateTableSparkAction.execute(); + return new InternalRow[] {newInternalRow(result.migratedDataFilesCount())}; } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/ProcedureInput.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/ProcedureInput.java index 42e4d8ba0603..0be4b38de79c 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/ProcedureInput.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/ProcedureInput.java @@ -68,6 +68,18 @@ public Boolean asBoolean(ProcedureParameter param, Boolean defaultValue) { return args.isNullAt(ordinal) ? defaultValue : (Boolean) args.getBoolean(ordinal); } + public Integer asInt(ProcedureParameter param) { + Integer value = asInt(param, null); + Preconditions.checkArgument(value != null, "Parameter '%s' is not set", param.name()); + return value; + } + + public Integer asInt(ProcedureParameter param, Integer defaultValue) { + validateParamType(param, DataTypes.IntegerType); + int ordinal = ordinal(param); + return args.isNullAt(ordinal) ? defaultValue : (Integer) args.getInt(ordinal); + } + public long asLong(ProcedureParameter param) { Long value = asLong(param, null); Preconditions.checkArgument(value != null, "Parameter '%s' is not set", param.name()); diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/SnapshotTableProcedure.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/SnapshotTableProcedure.java index 7a015a51e8ed..f709f64ebf62 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/SnapshotTableProcedure.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/SnapshotTableProcedure.java @@ -38,7 +38,8 @@ class SnapshotTableProcedure extends BaseProcedure { ProcedureParameter.required("source_table", DataTypes.StringType), ProcedureParameter.required("table", DataTypes.StringType), ProcedureParameter.optional("location", DataTypes.StringType), - ProcedureParameter.optional("properties", STRING_MAP) + ProcedureParameter.optional("properties", STRING_MAP), + ProcedureParameter.optional("parallelism", DataTypes.IntegerType) }; private static final StructType OUTPUT_TYPE = @@ -102,6 +103,12 @@ public InternalRow[] call(InternalRow args) { action.tableLocation(snapshotLocation); } + if (!args.isNullAt(4)) { + int parallelism = args.getInt(4); + Preconditions.checkArgument(parallelism > 0, "Parallelism should be larger than 0"); + action = action.executeWith(executorService(parallelism, "table-snapshot")); + } + SnapshotTable.Result result = action.tableProperties(properties).execute(); return new InternalRow[] {newInternalRow(result.importedDataFilesCount())}; } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java index 6372edde0782..2ff4f9156273 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java @@ -48,6 +48,10 @@ import org.apache.spark.sql.types.StructType; import org.apache.spark.unsafe.types.UTF8String; +/** + * @deprecated since 1.7.0, will be removed in 1.8.0; use {@link SparkFileWriterFactory} instead. + */ +@Deprecated class SparkAppenderFactory implements FileAppenderFactory { private final Map properties; private final Schema writeSchema; diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java index afb0f434aa41..5634e1436081 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java @@ -48,7 +48,6 @@ import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.Spark3Util; @@ -243,8 +242,7 @@ public boolean pushAggregation(Aggregation aggregation) { scan = scan.filter(filterExpression()); try (CloseableIterable fileScanTasks = scan.planFiles()) { - List tasks = ImmutableList.copyOf(fileScanTasks); - for (FileScanTask task : tasks) { + for (FileScanTask task : fileScanTasks) { if (!task.deletes().isEmpty()) { LOG.info("Skipping aggregate pushdown: detected row level deletes"); return false; diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java index 028d495b894d..06f118be0c6f 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java @@ -183,6 +183,11 @@ public void before() { public void after() throws IOException { // Drop the hive table. spark.sql(String.format("DROP TABLE IF EXISTS %s", baseTableName)); + spark.sessionState().catalogManager().reset(); + spark.conf().unset("spark.sql.catalog.spark_catalog.type"); + spark.conf().unset("spark.sql.catalog.spark_catalog.default-namespace"); + spark.conf().unset("spark.sql.catalog.spark_catalog.parquet-enabled"); + spark.conf().unset("spark.sql.catalog.spark_catalog.cache-enabled"); } @Test @@ -728,6 +733,8 @@ public void testStructOfThreeLevelLists() throws Exception { @Test public void testTwoLevelList() throws IOException { + Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", true); String tableName = sourceName("testTwoLevelList"); @@ -811,6 +818,8 @@ public boolean accept(File dir, String name) { } private void threeLevelList(boolean useLegacyMode) throws Exception { + Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); String tableName = sourceName(String.format("threeLevelList_%s", useLegacyMode)); @@ -833,6 +842,8 @@ private void threeLevelList(boolean useLegacyMode) throws Exception { } private void threeLevelListWithNestedStruct(boolean useLegacyMode) throws Exception { + Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); String tableName = @@ -858,6 +869,8 @@ private void threeLevelListWithNestedStruct(boolean useLegacyMode) throws Except } private void threeLevelLists(boolean useLegacyMode) throws Exception { + Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); String tableName = sourceName(String.format("threeLevelLists_%s", useLegacyMode)); @@ -885,6 +898,8 @@ private void threeLevelLists(boolean useLegacyMode) throws Exception { } private void structOfThreeLevelLists(boolean useLegacyMode) throws Exception { + Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); String tableName = sourceName(String.format("structOfThreeLevelLists_%s", useLegacyMode)); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateTableAction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateTableAction.java new file mode 100644 index 000000000000..7bed72b7cc2c --- /dev/null +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateTableAction.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.actions; + +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; + +public class TestMigrateTableAction extends SparkCatalogTestBase { + + public TestMigrateTableAction( + String catalogName, String implementation, Map config) { + super(catalogName, implementation, config); + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); + } + + @After + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS %s_BACKUP_", tableName); + } + + @Test + public void testMigrateWithParallelTasks() throws IOException { + String location = temp.newFolder().toURI().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + tableName, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); + + AtomicInteger migrationThreadsIndex = new AtomicInteger(0); + SparkActions.get() + .migrateTable(tableName) + .executeWith( + Executors.newFixedThreadPool( + 4, + runnable -> { + Thread thread = new Thread(runnable); + thread.setName("table-migration-" + migrationThreadsIndex.getAndIncrement()); + thread.setDaemon(true); + return thread; + })) + .execute(); + Assert.assertEquals(migrationThreadsIndex.get(), 2); + } +} diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java new file mode 100644 index 000000000000..8e6358f51bcd --- /dev/null +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.actions; + +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; + +public class TestSnapshotTableAction extends SparkCatalogTestBase { + private static final String SOURCE_NAME = "spark_catalog.default.source"; + + public TestSnapshotTableAction( + String catalogName, String implementation, Map config) { + super(catalogName, implementation, config); + } + + @After + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS %s PURGE", SOURCE_NAME); + } + + @Test + public void testSnapshotWithParallelTasks() throws IOException { + String location = temp.newFolder().toURI().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", SOURCE_NAME); + + AtomicInteger snapshotThreadsIndex = new AtomicInteger(0); + SparkActions.get() + .snapshotTable(SOURCE_NAME) + .as(tableName) + .executeWith( + Executors.newFixedThreadPool( + 4, + runnable -> { + Thread thread = new Thread(runnable); + thread.setName("table-snapshot-" + snapshotThreadsIndex.getAndIncrement()); + thread.setDaemon(true); + return thread; + })) + .execute(); + Assert.assertEquals(snapshotThreadsIndex.get(), 2); + } +} diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index 32b96a580da5..12011b66a5be 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -2021,8 +2021,13 @@ public void testFilesTablePartitionId() throws Exception { .save(loadLocation(tableIdentifier)); List actual = - spark.read().format("iceberg").load(loadLocation(tableIdentifier, "files")) - .sort(DataFile.SPEC_ID.name()).collectAsList().stream() + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "files")) + .sort(DataFile.SPEC_ID.name()) + .collectAsList() + .stream() .map(r -> (Integer) r.getAs(DataFile.SPEC_ID.name())) .collect(Collectors.toList()); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java index 76a4143fcb2b..bde87778ad62 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java @@ -37,6 +37,7 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.Files; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Parameter; @@ -98,18 +99,16 @@ public class TestSparkReaderDeletes extends DeleteReadTests { protected static SparkSession spark = null; protected static HiveCatalog catalog = null; - @Parameter private String format; - @Parameter(index = 1) private boolean vectorized; @Parameters(name = "format = {0}, vectorized = {1}") public static Object[][] parameters() { return new Object[][] { - new Object[] {"parquet", false}, - new Object[] {"parquet", true}, - new Object[] {"orc", false}, - new Object[] {"avro", false} + new Object[] {FileFormat.PARQUET, false}, + new Object[] {FileFormat.PARQUET, true}, + new Object[] {FileFormat.ORC, false}, + new Object[] {FileFormat.AVRO, false} }; } @@ -163,14 +162,14 @@ protected Table createTable(String name, Schema schema, PartitionSpec spec) { TableOperations ops = ((BaseTable) table).operations(); TableMetadata meta = ops.current(); ops.commit(meta, meta.upgradeToFormatVersion(2)); - table.updateProperties().set(TableProperties.DEFAULT_FILE_FORMAT, format).commit(); - if (format.equals("parquet") || format.equals("orc")) { + table.updateProperties().set(TableProperties.DEFAULT_FILE_FORMAT, format.name()).commit(); + if (format.equals(FileFormat.PARQUET) || format.equals(FileFormat.ORC)) { String vectorizationEnabled = - format.equals("parquet") + format.equals(FileFormat.PARQUET) ? TableProperties.PARQUET_VECTORIZATION_ENABLED : TableProperties.ORC_VECTORIZATION_ENABLED; String batchSize = - format.equals("parquet") + format.equals(FileFormat.PARQUET) ? TableProperties.PARQUET_BATCH_SIZE : TableProperties.ORC_BATCH_SIZE; table.updateProperties().set(vectorizationEnabled, String.valueOf(vectorized)).commit(); diff --git a/spark/v3.4/build.gradle b/spark/v3.4/build.gradle index 3b1761d39f63..a34ea6b92826 100644 --- a/spark/v3.4/build.gradle +++ b/spark/v3.4/build.gradle @@ -59,6 +59,7 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { implementation project(':iceberg-parquet') implementation project(':iceberg-arrow') implementation("org.scala-lang.modules:scala-collection-compat_${scalaVersion}:${libs.versions.scala.collection.compat.get()}") + implementation("org.apache.datasketches:datasketches-java:${libs.versions.datasketches.get()}") if (scalaVersion == '2.12') { // scala-collection-compat_2.12 pulls scala 2.12.17 and we need 2.12.18 for JDK 21 support implementation 'org.scala-lang:scala-library:2.12.18' @@ -190,7 +191,7 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer } project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersion}") { - apply plugin: 'io.github.goooler.shadow' + apply plugin: 'com.gradleup.shadow' tasks.jar.dependsOn tasks.shadowJar @@ -289,6 +290,7 @@ project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersio relocate 'com.carrotsearch', 'org.apache.iceberg.shaded.com.carrotsearch' relocate 'org.threeten.extra', 'org.apache.iceberg.shaded.org.threeten.extra' relocate 'org.roaringbitmap', 'org.apache.iceberg.shaded.org.roaringbitmap' + relocate 'org.apache.datasketches', 'org.apache.iceberg.shaded.org.apache.datasketches' archiveClassifier.set(null) } diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java index 5f995a7776c3..ae2062691d77 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java @@ -935,6 +935,28 @@ public void testPartitionedImportFromEmptyPartitionDoesNotThrow() { sql("SELECT * FROM %s ORDER BY id", tableName)); } + @Test + public void testAddFilesWithParallelism() { + createUnpartitionedHiveTable(); + + String createIceberg = + "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg"; + + sql(createIceberg, tableName); + + List result = + sql( + "CALL %s.system.add_files(table => '%s', source_table => '%s', parallelism => 2)", + catalogName, tableName, sourceTableName); + + assertEquals("Procedure output must match", ImmutableList.of(row(2L, 1L)), result); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT * FROM %s ORDER BY id", sourceTableName), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + private static final List EMPTY_QUERY_RESULT = Lists.newArrayList(); private static final StructField[] STRUCT = { diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java index 82df7567fce6..1cfe1ecf83be 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java @@ -20,6 +20,8 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.sql.Timestamp; +import java.time.Instant; import java.time.LocalDateTime; import java.util.List; import java.util.Map; @@ -252,6 +254,39 @@ public void testRollbackToTimestampWithoutExplicitCatalog() { sql("SELECT * FROM %s ORDER BY id", tableName)); } + @Test + public void testRollbackToTimestampBeforeOrEqualToOldestSnapshot() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot firstSnapshot = table.currentSnapshot(); + Timestamp beforeFirstSnapshot = + Timestamp.from(Instant.ofEpochMilli(firstSnapshot.timestampMillis() - 1)); + Timestamp exactFirstSnapshot = + Timestamp.from(Instant.ofEpochMilli(firstSnapshot.timestampMillis())); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rollback_to_timestamp(timestamp => TIMESTAMP '%s', table => '%s')", + catalogName, beforeFirstSnapshot, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot roll back, no valid snapshot older than: %s", + beforeFirstSnapshot.toInstant().toEpochMilli()); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rollback_to_timestamp(timestamp => TIMESTAMP '%s', table => '%s')", + catalogName, exactFirstSnapshot, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot roll back, no valid snapshot older than: %s", + exactFirstSnapshot.toInstant().toEpochMilli()); + } + @Test public void testInvalidRollbackToTimestampCases() { String timestamp = "TIMESTAMP '2007-12-03T10:15:30'"; diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java index 905cb8fe07fb..421d6efc9389 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java @@ -35,6 +35,7 @@ public class TestSnapshotTableProcedure extends SparkExtensionsTestBase { private static final String SOURCE_NAME = "spark_catalog.default.source"; + // Currently we can only Snapshot only out of the Spark Session Catalog public TestSnapshotTableProcedure( diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java index 65b8669c35a0..1380711ed7b7 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java @@ -23,6 +23,7 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Random; import java.util.stream.Collectors; @@ -92,7 +93,7 @@ public TestViews(String catalog, String implementation, Map prop @Test public void readFromView() throws NoSuchTableException { insertRows(10); - String viewName = "simpleView"; + String viewName = viewName("simpleView"); String sql = String.format("SELECT id FROM %s", tableName); ViewCatalog viewCatalog = viewCatalog(); @@ -118,7 +119,7 @@ public void readFromView() throws NoSuchTableException { @Test public void readFromTrinoView() throws NoSuchTableException { insertRows(10); - String viewName = "trinoView"; + String viewName = viewName("trinoView"); String sql = String.format("SELECT id FROM %s", tableName); ViewCatalog viewCatalog = viewCatalog(); @@ -143,8 +144,8 @@ public void readFromTrinoView() throws NoSuchTableException { @Test public void readFromMultipleViews() throws NoSuchTableException { insertRows(6); - String viewName = "firstView"; - String secondView = "secondView"; + String viewName = viewName("firstView"); + String secondView = viewName("secondView"); String viewSQL = String.format("SELECT id FROM %s WHERE id <= 3", tableName); String secondViewSQL = String.format("SELECT id FROM %s WHERE id > 3", tableName); @@ -176,7 +177,7 @@ public void readFromMultipleViews() throws NoSuchTableException { @Test public void readFromViewUsingNonExistingTable() throws NoSuchTableException { insertRows(10); - String viewName = "viewWithNonExistingTable"; + String viewName = viewName("viewWithNonExistingTable"); ViewCatalog viewCatalog = viewCatalog(); Schema schema = new Schema(Types.NestedField.required(1, "id", Types.LongType.get())); @@ -200,7 +201,7 @@ public void readFromViewUsingNonExistingTable() throws NoSuchTableException { @Test public void readFromViewUsingNonExistingTableColumn() throws NoSuchTableException { insertRows(10); - String viewName = "viewWithNonExistingColumn"; + String viewName = viewName("viewWithNonExistingColumn"); ViewCatalog viewCatalog = viewCatalog(); Schema schema = new Schema(Types.NestedField.required(1, "non_existing", Types.LongType.get())); @@ -222,7 +223,7 @@ public void readFromViewUsingNonExistingTableColumn() throws NoSuchTableExceptio @Test public void readFromViewUsingInvalidSQL() throws NoSuchTableException { insertRows(10); - String viewName = "viewWithInvalidSQL"; + String viewName = viewName("viewWithInvalidSQL"); ViewCatalog viewCatalog = viewCatalog(); Schema schema = tableCatalog().loadTable(TableIdentifier.of(NAMESPACE, tableName)).schema(); @@ -244,7 +245,7 @@ public void readFromViewUsingInvalidSQL() throws NoSuchTableException { @Test public void readFromViewWithStaleSchema() throws NoSuchTableException { insertRows(10); - String viewName = "staleView"; + String viewName = viewName("staleView"); String sql = String.format("SELECT id, data FROM %s", tableName); ViewCatalog viewCatalog = viewCatalog(); @@ -270,7 +271,7 @@ public void readFromViewWithStaleSchema() throws NoSuchTableException { @Test public void readFromViewHiddenByTempView() throws NoSuchTableException { insertRows(10); - String viewName = "viewHiddenByTempView"; + String viewName = viewName("viewHiddenByTempView"); ViewCatalog viewCatalog = viewCatalog(); Schema schema = tableCatalog().loadTable(TableIdentifier.of(NAMESPACE, tableName)).schema(); @@ -297,7 +298,7 @@ public void readFromViewHiddenByTempView() throws NoSuchTableException { @Test public void readFromViewWithGlobalTempView() throws NoSuchTableException { insertRows(10); - String viewName = "viewWithGlobalTempView"; + String viewName = viewName("viewWithGlobalTempView"); String sql = String.format("SELECT id FROM %s WHERE id > 5", tableName); ViewCatalog viewCatalog = viewCatalog(); @@ -327,8 +328,8 @@ public void readFromViewWithGlobalTempView() throws NoSuchTableException { @Test public void readFromViewReferencingAnotherView() throws NoSuchTableException { insertRows(10); - String firstView = "viewBeingReferencedInAnotherView"; - String viewReferencingOtherView = "viewReferencingOtherView"; + String firstView = viewName("viewBeingReferencedInAnotherView"); + String viewReferencingOtherView = viewName("viewReferencingOtherView"); String firstSQL = String.format("SELECT id FROM %s WHERE id <= 5", tableName); String secondSQL = String.format("SELECT id FROM %s WHERE id > 4", firstView); @@ -358,8 +359,8 @@ public void readFromViewReferencingAnotherView() throws NoSuchTableException { @Test public void readFromViewReferencingTempView() throws NoSuchTableException { insertRows(10); - String tempView = "tempViewBeingReferencedInAnotherView"; - String viewReferencingTempView = "viewReferencingTempView"; + String tempView = viewName("tempViewBeingReferencedInAnotherView"); + String viewReferencingTempView = viewName("viewReferencingTempView"); String sql = String.format("SELECT id FROM %s", tempView); ViewCatalog viewCatalog = viewCatalog(); @@ -394,8 +395,8 @@ public void readFromViewReferencingTempView() throws NoSuchTableException { @Test public void readFromViewReferencingAnotherViewHiddenByTempView() throws NoSuchTableException { insertRows(10); - String innerViewName = "inner_view"; - String outerViewName = "outer_view"; + String innerViewName = viewName("inner_view"); + String outerViewName = viewName("outer_view"); String innerViewSQL = String.format("SELECT * FROM %s WHERE id > 5", tableName); String outerViewSQL = String.format("SELECT id FROM %s", innerViewName); @@ -442,8 +443,8 @@ public void readFromViewReferencingAnotherViewHiddenByTempView() throws NoSuchTa @Test public void readFromViewReferencingGlobalTempView() throws NoSuchTableException { insertRows(10); - String globalTempView = "globalTempViewBeingReferenced"; - String viewReferencingTempView = "viewReferencingGlobalTempView"; + String globalTempView = viewName("globalTempViewBeingReferenced"); + String viewReferencingTempView = viewName("viewReferencingGlobalTempView"); ViewCatalog viewCatalog = viewCatalog(); Schema schema = tableCatalog().loadTable(TableIdentifier.of(NAMESPACE, tableName)).schema(); @@ -481,7 +482,7 @@ public void readFromViewReferencingGlobalTempView() throws NoSuchTableException public void readFromViewReferencingTempFunction() throws NoSuchTableException { insertRows(10); String viewName = viewName("viewReferencingTempFunction"); - String functionName = "test_avg"; + String functionName = viewName("test_avg"); String sql = String.format("SELECT %s(id) FROM %s", functionName, tableName); sql( "CREATE TEMPORARY FUNCTION %s AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage'", @@ -512,7 +513,7 @@ public void readFromViewReferencingTempFunction() throws NoSuchTableException { @Test public void readFromViewWithCTE() throws NoSuchTableException { insertRows(10); - String viewName = "viewWithCTE"; + String viewName = viewName("viewWithCTE"); String sql = String.format( "WITH max_by_data AS (SELECT max(id) as max FROM %s) " @@ -534,7 +535,7 @@ public void readFromViewWithCTE() throws NoSuchTableException { @Test public void rewriteFunctionIdentifier() { - String viewName = "rewriteFunctionIdentifier"; + String viewName = viewName("rewriteFunctionIdentifier"); String sql = "SELECT iceberg_version() AS version"; assertThatThrownBy(() -> sql(sql)) @@ -560,7 +561,7 @@ public void rewriteFunctionIdentifier() { @Test public void builtinFunctionIdentifierNotRewritten() { - String viewName = "builtinFunctionIdentifierNotRewritten"; + String viewName = viewName("builtinFunctionIdentifierNotRewritten"); String sql = "SELECT trim(' abc ') AS result"; ViewCatalog viewCatalog = viewCatalog(); @@ -579,7 +580,7 @@ public void builtinFunctionIdentifierNotRewritten() { @Test public void rewriteFunctionIdentifierWithNamespace() { - String viewName = "rewriteFunctionIdentifierWithNamespace"; + String viewName = viewName("rewriteFunctionIdentifierWithNamespace"); String sql = "SELECT system.bucket(100, 'a') AS bucket_result, 'a' AS value"; ViewCatalog viewCatalog = viewCatalog(); @@ -606,7 +607,7 @@ public void rewriteFunctionIdentifierWithNamespace() { @Test public void fullFunctionIdentifier() { - String viewName = "fullFunctionIdentifier"; + String viewName = viewName("fullFunctionIdentifier"); String sql = String.format( "SELECT %s.system.bucket(100, 'a') AS bucket_result, 'a' AS value", catalogName); @@ -630,7 +631,7 @@ public void fullFunctionIdentifier() { @Test public void fullFunctionIdentifierNotRewrittenLoadFailure() { - String viewName = "fullFunctionIdentifierNotRewrittenLoadFailure"; + String viewName = viewName("fullFunctionIdentifierNotRewrittenLoadFailure"); String sql = "SELECT spark_catalog.system.bucket(100, 'a') AS bucket_result, 'a' AS value"; // avoid namespace failures @@ -822,7 +823,7 @@ public void renameViewTargetAlreadyExistsAsTable() { @Test public void dropView() { - String viewName = "viewToBeDropped"; + String viewName = viewName("viewToBeDropped"); String sql = String.format("SELECT id FROM %s", tableName); ViewCatalog viewCatalog = viewCatalog(); @@ -851,7 +852,7 @@ public void dropNonExistingView() { @Test public void dropViewIfExists() { - String viewName = "viewToBeDropped"; + String viewName = viewName("viewToBeDropped"); String sql = String.format("SELECT id FROM %s", tableName); ViewCatalog viewCatalog = viewCatalog(); @@ -876,7 +877,7 @@ public void dropViewIfExists() { /** The purpose of this test is mainly to make sure that normal view deletion isn't messed up */ @Test public void dropGlobalTempView() { - String globalTempView = "globalViewToBeDropped"; + String globalTempView = viewName("globalViewToBeDropped"); sql("CREATE GLOBAL TEMPORARY VIEW %s AS SELECT id FROM %s", globalTempView, tableName); assertThat(v1SessionCatalog().getGlobalTempView(globalTempView).isDefined()).isTrue(); @@ -887,7 +888,7 @@ public void dropGlobalTempView() { /** The purpose of this test is mainly to make sure that normal view deletion isn't messed up */ @Test public void dropTempView() { - String tempView = "tempViewToBeDropped"; + String tempView = viewName("tempViewToBeDropped"); sql("CREATE TEMPORARY VIEW %s AS SELECT id FROM %s", tempView, tableName); assertThat(v1SessionCatalog().getTempView(tempView).isDefined()).isTrue(); @@ -898,7 +899,7 @@ public void dropTempView() { /** The purpose of this test is mainly to make sure that normal view deletion isn't messed up */ @Test public void dropV1View() { - String v1View = "v1ViewToBeDropped"; + String v1View = viewName("v1ViewToBeDropped"); sql("USE spark_catalog"); sql("CREATE NAMESPACE IF NOT EXISTS %s", NAMESPACE); sql("CREATE TABLE %s (id INT, data STRING)", tableName); @@ -929,7 +930,7 @@ private String viewName(String viewName) { @Test public void createViewIfNotExists() { - String viewName = "viewThatAlreadyExists"; + String viewName = viewName("viewThatAlreadyExists"); sql("CREATE VIEW %s AS SELECT id FROM %s", viewName, tableName); assertThatThrownBy(() -> sql("CREATE VIEW %s AS SELECT id FROM %s", viewName, tableName)) @@ -970,8 +971,8 @@ public void createViewWithInvalidSQL() { @Test public void createViewReferencingTempView() throws NoSuchTableException { insertRows(10); - String tempView = "temporaryViewBeingReferencedInAnotherView"; - String viewReferencingTempView = "viewReferencingTemporaryView"; + String tempView = viewName("temporaryViewBeingReferencedInAnotherView"); + String viewReferencingTempView = viewName("viewReferencingTemporaryView"); sql("CREATE TEMPORARY VIEW %s AS SELECT id FROM %s WHERE id <= 5", tempView, tableName); @@ -989,8 +990,8 @@ public void createViewReferencingTempView() throws NoSuchTableException { @Test public void createViewReferencingGlobalTempView() throws NoSuchTableException { insertRows(10); - String globalTempView = "globalTemporaryViewBeingReferenced"; - String viewReferencingTempView = "viewReferencingGlobalTemporaryView"; + String globalTempView = viewName("globalTemporaryViewBeingReferenced"); + String viewReferencingTempView = viewName("viewReferencingGlobalTemporaryView"); sql( "CREATE GLOBAL TEMPORARY VIEW %s AS SELECT id FROM %s WHERE id <= 5", @@ -1013,7 +1014,7 @@ public void createViewReferencingGlobalTempView() throws NoSuchTableException { @Test public void createViewReferencingTempFunction() { String viewName = viewName("viewReferencingTemporaryFunction"); - String functionName = "test_avg_func"; + String functionName = viewName("test_avg_func"); sql( "CREATE TEMPORARY FUNCTION %s AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage'", @@ -1032,7 +1033,7 @@ public void createViewReferencingTempFunction() { @Test public void createViewReferencingQualifiedTempFunction() { String viewName = viewName("viewReferencingTemporaryFunction"); - String functionName = "test_avg_func_qualified"; + String functionName = viewName("test_avg_func_qualified"); sql( "CREATE TEMPORARY FUNCTION %s AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage'", @@ -1070,7 +1071,7 @@ public void createViewUsingNonExistingTable() { @Test public void createViewWithMismatchedColumnCounts() { - String viewName = "viewWithMismatchedColumnCounts"; + String viewName = viewName("viewWithMismatchedColumnCounts"); assertThatThrownBy( () -> sql("CREATE VIEW %s (id, data) AS SELECT id FROM %s", viewName, tableName)) @@ -1094,7 +1095,7 @@ public void createViewWithMismatchedColumnCounts() { @Test public void createViewWithColumnAliases() throws NoSuchTableException { insertRows(6); - String viewName = "viewWithColumnAliases"; + String viewName = viewName("viewWithColumnAliases"); sql( "CREATE VIEW %s (new_id COMMENT 'ID', new_data COMMENT 'DATA') AS SELECT id, data FROM %s WHERE id <= 3", @@ -1141,7 +1142,7 @@ public void createViewWithDuplicateColumnNames() { @Test public void createViewWithDuplicateQueryColumnNames() throws NoSuchTableException { insertRows(3); - String viewName = "viewWithDuplicateQueryColumnNames"; + String viewName = viewName("viewWithDuplicateQueryColumnNames"); String sql = String.format("SELECT id, id FROM %s WHERE id <= 3", tableName); // not specifying column aliases in the view should fail @@ -1159,7 +1160,7 @@ public void createViewWithDuplicateQueryColumnNames() throws NoSuchTableExceptio @Test public void createViewWithCTE() throws NoSuchTableException { insertRows(10); - String viewName = "simpleViewWithCTE"; + String viewName = viewName("simpleViewWithCTE"); String sql = String.format( "WITH max_by_data AS (SELECT max(id) as max FROM %s) " @@ -1174,8 +1175,8 @@ public void createViewWithCTE() throws NoSuchTableException { @Test public void createViewWithConflictingNamesForCTEAndTempView() throws NoSuchTableException { insertRows(10); - String viewName = "viewWithConflictingNamesForCTEAndTempView"; - String cteName = "cteName"; + String viewName = viewName("viewWithConflictingNamesForCTEAndTempView"); + String cteName = viewName("cteName"); String sql = String.format( "WITH %s AS (SELECT max(id) as max FROM %s) " @@ -1192,8 +1193,8 @@ public void createViewWithConflictingNamesForCTEAndTempView() throws NoSuchTable @Test public void createViewWithCTEReferencingTempView() { - String viewName = "viewWithCTEReferencingTempView"; - String tempViewInCTE = "tempViewInCTE"; + String viewName = viewName("viewWithCTEReferencingTempView"); + String tempViewInCTE = viewName("tempViewInCTE"); String sql = String.format( "WITH max_by_data AS (SELECT max(id) as max FROM %s) " @@ -1212,8 +1213,8 @@ public void createViewWithCTEReferencingTempView() { @Test public void createViewWithCTEReferencingTempFunction() { - String viewName = "viewWithCTEReferencingTempFunction"; - String functionName = "avg_function_in_cte"; + String viewName = viewName("viewWithCTEReferencingTempFunction"); + String functionName = viewName("avg_function_in_cte"); String sql = String.format( "WITH avg_data AS (SELECT %s(id) as avg FROM %s) " @@ -1246,8 +1247,8 @@ public void createViewWithNonExistingQueryColumn() { @Test public void createViewWithSubqueryExpressionUsingTempView() { - String viewName = "viewWithSubqueryExpression"; - String tempView = "simpleTempView"; + String viewName = viewName("viewWithSubqueryExpression"); + String tempView = viewName("simpleTempView"); String sql = String.format("SELECT * FROM %s WHERE id = (SELECT id FROM %s)", tableName, tempView); @@ -1263,8 +1264,8 @@ public void createViewWithSubqueryExpressionUsingTempView() { @Test public void createViewWithSubqueryExpressionUsingGlobalTempView() { - String viewName = "simpleViewWithSubqueryExpression"; - String globalTempView = "simpleGlobalTempView"; + String viewName = viewName("simpleViewWithSubqueryExpression"); + String globalTempView = viewName("simpleGlobalTempView"); String sql = String.format( "SELECT * FROM %s WHERE id = (SELECT id FROM global_temp.%s)", @@ -1285,7 +1286,7 @@ public void createViewWithSubqueryExpressionUsingGlobalTempView() { @Test public void createViewWithSubqueryExpressionUsingTempFunction() { String viewName = viewName("viewWithSubqueryExpression"); - String functionName = "avg_function_in_subquery"; + String functionName = viewName("avg_function_in_subquery"); String sql = String.format( "SELECT * FROM %s WHERE id < (SELECT %s(id) FROM %s)", @@ -1355,7 +1356,7 @@ public void createViewWithSubqueryExpressionInQueryThatIsRewritten() throws NoSu @Test public void describeView() { - String viewName = "describeView"; + String viewName = viewName("describeView"); sql("CREATE VIEW %s AS SELECT id, data FROM %s WHERE id <= 3", viewName, tableName); assertThat(sql("DESCRIBE %s", viewName)) @@ -1364,7 +1365,7 @@ public void describeView() { @Test public void describeExtendedView() { - String viewName = "describeExtendedView"; + String viewName = viewName("describeExtendedView"); String sql = String.format("SELECT id, data FROM %s WHERE id <= 3", tableName); sql( @@ -1389,7 +1390,7 @@ public void describeExtendedView() { @Test public void showViewProperties() { - String viewName = "showViewProps"; + String viewName = viewName("showViewProps"); sql( "CREATE VIEW %s TBLPROPERTIES ('key1'='val1', 'key2'='val2') AS SELECT id, data FROM %s WHERE id <= 3", @@ -1400,7 +1401,7 @@ public void showViewProperties() { @Test public void showViewPropertiesByKey() { - String viewName = "showViewPropsByKey"; + String viewName = viewName("showViewPropsByKey"); sql("CREATE VIEW %s AS SELECT id, data FROM %s WHERE id <= 3", viewName, tableName); assertThat(sql("SHOW TBLPROPERTIES %s", viewName)).contains(row("provider", "iceberg")); @@ -1421,39 +1422,43 @@ public void showViewPropertiesByKey() { public void showViews() throws NoSuchTableException { insertRows(6); String sql = String.format("SELECT * from %s", tableName); - sql("CREATE VIEW v1 AS %s", sql); - sql("CREATE VIEW prefixV2 AS %s", sql); - sql("CREATE VIEW prefixV3 AS %s", sql); - sql("CREATE GLOBAL TEMPORARY VIEW globalViewForListing AS %s", sql); - sql("CREATE TEMPORARY VIEW tempViewForListing AS %s", sql); + String v1 = viewName("v1"); + String prefixV2 = viewName("prefixV2"); + String prefixV3 = viewName("prefixV3"); + String globalViewForListing = viewName("globalViewForListing"); + String tempViewForListing = viewName("tempViewForListing"); + sql("CREATE VIEW %s AS %s", v1, sql); + sql("CREATE VIEW %s AS %s", prefixV2, sql); + sql("CREATE VIEW %s AS %s", prefixV3, sql); + sql("CREATE GLOBAL TEMPORARY VIEW %s AS %s", globalViewForListing, sql); + sql("CREATE TEMPORARY VIEW %s AS %s", tempViewForListing, sql); // spark stores temp views case-insensitive by default - Object[] tempView = row("", "tempviewforlisting", true); + Object[] tempView = row("", tempViewForListing.toLowerCase(Locale.ROOT), true); assertThat(sql("SHOW VIEWS")) .contains( - row(NAMESPACE.toString(), "prefixV2", false), - row(NAMESPACE.toString(), "prefixV3", false), - row(NAMESPACE.toString(), "v1", false), + row(NAMESPACE.toString(), prefixV2, false), + row(NAMESPACE.toString(), prefixV3, false), + row(NAMESPACE.toString(), v1, false), tempView); assertThat(sql("SHOW VIEWS IN %s", catalogName)) .contains( - row(NAMESPACE.toString(), "prefixV2", false), - row(NAMESPACE.toString(), "prefixV3", false), - row(NAMESPACE.toString(), "v1", false), + row(NAMESPACE.toString(), prefixV2, false), + row(NAMESPACE.toString(), prefixV3, false), + row(NAMESPACE.toString(), v1, false), tempView); assertThat(sql("SHOW VIEWS IN %s.%s", catalogName, NAMESPACE)) .contains( - row(NAMESPACE.toString(), "prefixV2", false), - row(NAMESPACE.toString(), "prefixV3", false), - row(NAMESPACE.toString(), "v1", false), + row(NAMESPACE.toString(), prefixV2, false), + row(NAMESPACE.toString(), prefixV3, false), + row(NAMESPACE.toString(), v1, false), tempView); assertThat(sql("SHOW VIEWS LIKE 'pref*'")) .contains( - row(NAMESPACE.toString(), "prefixV2", false), - row(NAMESPACE.toString(), "prefixV3", false)); + row(NAMESPACE.toString(), prefixV2, false), row(NAMESPACE.toString(), prefixV3, false)); assertThat(sql("SHOW VIEWS LIKE 'non-existing'")).isEmpty(); @@ -1462,7 +1467,7 @@ public void showViews() throws NoSuchTableException { assertThat(sql("SHOW VIEWS IN global_temp")) .contains( // spark stores temp views case-insensitive by default - row("global_temp", "globalviewforlisting", true), tempView); + row("global_temp", globalViewForListing.toLowerCase(Locale.ROOT), true), tempView); sql("USE spark_catalog"); assertThat(sql("SHOW VIEWS")).contains(tempView); @@ -1503,7 +1508,7 @@ public void showViewsWithCurrentNamespace() { @Test public void showCreateSimpleView() { - String viewName = "showCreateSimpleView"; + String viewName = viewName("showCreateSimpleView"); String sql = String.format("SELECT id, data FROM %s WHERE id <= 3", tableName); sql("CREATE VIEW %s AS %s", viewName, sql); @@ -1524,7 +1529,7 @@ public void showCreateSimpleView() { @Test public void showCreateComplexView() { - String viewName = "showCreateComplexView"; + String viewName = viewName("showCreateComplexView"); String sql = String.format("SELECT id, data FROM %s WHERE id <= 3", tableName); sql( @@ -1551,7 +1556,7 @@ public void showCreateComplexView() { @Test public void alterViewSetProperties() { - String viewName = "viewWithSetProperties"; + String viewName = viewName("viewWithSetProperties"); sql("CREATE VIEW %s AS SELECT id FROM %s WHERE id <= 3", viewName, tableName); @@ -1573,7 +1578,7 @@ public void alterViewSetProperties() { @Test public void alterViewSetReservedProperties() { - String viewName = "viewWithSetReservedProperties"; + String viewName = viewName("viewWithSetReservedProperties"); sql("CREATE VIEW %s AS SELECT id FROM %s WHERE id <= 3", viewName, tableName); @@ -1604,7 +1609,7 @@ public void alterViewSetReservedProperties() { @Test public void alterViewUnsetProperties() { - String viewName = "viewWithUnsetProperties"; + String viewName = viewName("viewWithUnsetProperties"); sql("CREATE VIEW %s AS SELECT id FROM %s WHERE id <= 3", viewName, tableName); ViewCatalog viewCatalog = viewCatalog(); @@ -1625,7 +1630,7 @@ public void alterViewUnsetProperties() { @Test public void alterViewUnsetUnknownProperty() { - String viewName = "viewWithUnsetUnknownProp"; + String viewName = viewName("viewWithUnsetUnknownProp"); sql("CREATE VIEW %s AS SELECT id FROM %s WHERE id <= 3", viewName, tableName); assertThatThrownBy(() -> sql("ALTER VIEW %s UNSET TBLPROPERTIES ('unknown-key')", viewName)) @@ -1639,7 +1644,7 @@ public void alterViewUnsetUnknownProperty() { @Test public void alterViewUnsetReservedProperties() { - String viewName = "viewWithUnsetReservedProperties"; + String viewName = viewName("viewWithUnsetReservedProperties"); sql("CREATE VIEW %s AS SELECT id FROM %s WHERE id <= 3", viewName, tableName); @@ -1722,7 +1727,7 @@ public void createOrReplaceViewWithColumnAliases() throws NoSuchTableException { @Test public void alterViewIsNotSupported() throws NoSuchTableException { insertRows(6); - String viewName = "alteredView"; + String viewName = viewName("alteredView"); sql("CREATE VIEW %s AS SELECT id, data FROM %s WHERE id <= 3", viewName, tableName); diff --git a/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java b/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java index 59bc70854cbe..20be98d17bb2 100644 --- a/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java +++ b/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java @@ -103,7 +103,7 @@ public void testAlterTable() throws NoSuchTableException { sql( "CREATE TABLE %s (category int, id bigint, data string, ts timestamp) USING iceberg", tableName); - Table table = getTable(); + Table table; // Add examples sql("ALTER TABLE %s ADD PARTITION FIELD bucket(16, id)", tableName); sql("ALTER TABLE %s ADD PARTITION FIELD truncate(data, 4)", tableName); diff --git a/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java b/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java index 5a7df7283728..b6ade2bff3e1 100644 --- a/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java +++ b/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java @@ -22,6 +22,7 @@ import java.sql.Timestamp; import java.util.List; +import java.util.Locale; import java.util.UUID; import java.util.concurrent.TimeUnit; import org.apache.iceberg.AppendFiles; @@ -124,7 +125,7 @@ private void appendData() { for (int i = 0; i < NUM_SNAPSHOTS; i++) { AppendFiles appendFiles = table().newFastAppend(); for (int j = 0; j < NUM_FILES; j++) { - String path = String.format("%s/path/to/data-%d-%d.parquet", location, i, j); + String path = String.format(Locale.ROOT, "%s/path/to/data-%d-%d.parquet", location, i, j); validAndOrphanPaths.add(path); DataFile dataFile = DataFiles.builder(partitionSpec) diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java index f756c4cde015..99586f2503c2 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java @@ -52,6 +52,7 @@ public abstract class SparkContentFile implements ContentFile { private final int keyMetadataPosition; private final int splitOffsetsPosition; private final int sortOrderIdPosition; + private final int fileSpecIdPosition; private final int equalityIdsPosition; private final Type lowerBoundsType; private final Type upperBoundsType; @@ -100,6 +101,7 @@ public abstract class SparkContentFile implements ContentFile { this.keyMetadataPosition = positions.get(DataFile.KEY_METADATA.name()); this.splitOffsetsPosition = positions.get(DataFile.SPLIT_OFFSETS.name()); this.sortOrderIdPosition = positions.get(DataFile.SORT_ORDER_ID.name()); + this.fileSpecIdPosition = positions.get(DataFile.SPEC_ID.name()); this.equalityIdsPosition = positions.get(DataFile.EQUALITY_IDS.name()); } @@ -120,7 +122,10 @@ public Long pos() { @Override public int specId() { - return -1; + if (wrapped.isNullAt(fileSpecIdPosition)) { + return -1; + } + return wrapped.getAs(fileSpecIdPosition); } @Override diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java index 6f57c7ae376c..e103104171f6 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java @@ -28,6 +28,7 @@ import java.util.List; import java.util.Map; import java.util.UUID; +import java.util.concurrent.ExecutorService; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -277,7 +278,8 @@ private static List listPartition( PartitionSpec spec, SerializableConfiguration conf, MetricsConfig metricsConfig, - NameMapping mapping) { + NameMapping mapping, + int parallelism) { return TableMigrationUtil.listPartition( partition.values, partition.uri, @@ -285,7 +287,26 @@ private static List listPartition( spec, conf.get(), metricsConfig, - mapping); + mapping, + parallelism); + } + + private static List listPartition( + SparkPartition partition, + PartitionSpec spec, + SerializableConfiguration conf, + MetricsConfig metricsConfig, + NameMapping mapping, + ExecutorService service) { + return TableMigrationUtil.listPartition( + partition.values, + partition.uri, + partition.format, + spec, + conf.get(), + metricsConfig, + mapping, + service); } private static SparkPartition toSparkPartition( @@ -382,6 +403,114 @@ public static void importSparkTable( String stagingDir, Map partitionFilter, boolean checkDuplicateFiles) { + importSparkTable( + spark, sourceTableIdent, targetTable, stagingDir, partitionFilter, checkDuplicateFiles, 1); + } + + /** + * Import files from an existing Spark table to an Iceberg table. + * + *

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

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

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

The import uses the Spark session to get table metadata. It assumes no operation is going on + * the original and target table and thus is not thread-safe. + * + * @param spark a Spark session + * @param sourceTableIdent an identifier of the source Spark table + * @param targetTable an Iceberg table where to import the data + * @param stagingDir a staging directory to store temporary manifest files + * @param partitionFilter only import partitions whose values match those in the map, can be + * partially defined + * @param checkDuplicateFiles if true, throw exception if import results in a duplicate data file + * @param service executor service to use for file reading + */ + public static void importSparkTable( + SparkSession spark, + TableIdentifier sourceTableIdent, + Table targetTable, + String stagingDir, + Map partitionFilter, + boolean checkDuplicateFiles, + ExecutorService service) { SessionCatalog catalog = spark.sessionState().catalog(); String db = @@ -402,7 +531,7 @@ public static void importSparkTable( if (Objects.equal(spec, PartitionSpec.unpartitioned())) { importUnpartitionedSparkTable( - spark, sourceTableIdentWithDB, targetTable, checkDuplicateFiles); + spark, sourceTableIdentWithDB, targetTable, checkDuplicateFiles, service); } else { List sourceTablePartitions = getPartitions(spark, sourceTableIdent, partitionFilter); @@ -410,7 +539,13 @@ public static void importSparkTable( targetTable.newAppend().commit(); } else { importSparkPartitions( - spark, sourceTablePartitions, targetTable, spec, stagingDir, checkDuplicateFiles); + spark, + sourceTablePartitions, + targetTable, + spec, + stagingDir, + checkDuplicateFiles, + service); } } } catch (AnalysisException e) { @@ -443,7 +578,8 @@ public static void importSparkTable( targetTable, stagingDir, Collections.emptyMap(), - checkDuplicateFiles); + checkDuplicateFiles, + 1); } /** @@ -460,14 +596,15 @@ public static void importSparkTable( public static void importSparkTable( SparkSession spark, TableIdentifier sourceTableIdent, Table targetTable, String stagingDir) { importSparkTable( - spark, sourceTableIdent, targetTable, stagingDir, Collections.emptyMap(), false); + spark, sourceTableIdent, targetTable, stagingDir, Collections.emptyMap(), false, 1); } private static void importUnpartitionedSparkTable( SparkSession spark, TableIdentifier sourceTableIdent, Table targetTable, - boolean checkDuplicateFiles) { + boolean checkDuplicateFiles, + ExecutorService service) { try { CatalogTable sourceTable = spark.sessionState().catalog().getTableMetadata(sourceTableIdent); Option format = @@ -492,7 +629,8 @@ private static void importUnpartitionedSparkTable( spec, conf, metricsConfig, - nameMapping); + nameMapping, + service); if (checkDuplicateFiles) { Dataset importedFiles = @@ -540,9 +678,60 @@ public static void importSparkPartitions( PartitionSpec spec, String stagingDir, boolean checkDuplicateFiles) { + importSparkPartitions(spark, partitions, targetTable, spec, stagingDir, checkDuplicateFiles, 1); + } + + /** + * Import files from given partitions to an Iceberg table. + * + * @param spark a Spark session + * @param partitions partitions to import + * @param targetTable an Iceberg table where to import the data + * @param spec a partition spec + * @param stagingDir a staging directory to store temporary manifest files + * @param checkDuplicateFiles if true, throw exception if import results in a duplicate data file + * @param parallelism number of threads to use for file reading + */ + public static void importSparkPartitions( + SparkSession spark, + List partitions, + Table targetTable, + PartitionSpec spec, + String stagingDir, + boolean checkDuplicateFiles, + int parallelism) { + importSparkPartitions( + spark, + partitions, + targetTable, + spec, + stagingDir, + checkDuplicateFiles, + TableMigrationUtil.migrationService(parallelism)); + } + + /** + * Import files from given partitions to an Iceberg table. + * + * @param spark a Spark session + * @param partitions partitions to import + * @param targetTable an Iceberg table where to import the data + * @param spec a partition spec + * @param stagingDir a staging directory to store temporary manifest files + * @param checkDuplicateFiles if true, throw exception if import results in a duplicate data file + * @param service executor service to use for file reading + */ + public static void importSparkPartitions( + SparkSession spark, + List partitions, + Table targetTable, + PartitionSpec spec, + String stagingDir, + boolean checkDuplicateFiles, + ExecutorService service) { Configuration conf = spark.sessionState().newHadoopConf(); SerializableConfiguration serializableConf = new SerializableConfiguration(conf); - int parallelism = + int listingParallelism = Math.min( partitions.size(), spark.sessionState().conf().parallelPartitionDiscoveryParallelism()); int numShufflePartitions = spark.sessionState().conf().numShufflePartitions(); @@ -552,7 +741,7 @@ public static void importSparkPartitions( nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null; JavaSparkContext sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); - JavaRDD partitionRDD = sparkContext.parallelize(partitions, parallelism); + JavaRDD partitionRDD = sparkContext.parallelize(partitions, listingParallelism); Dataset partitionDS = spark.createDataset(partitionRDD.rdd(), Encoders.javaSerialization(SparkPartition.class)); @@ -562,7 +751,12 @@ public static void importSparkPartitions( (FlatMapFunction) sparkPartition -> listPartition( - sparkPartition, spec, serializableConf, metricsConfig, nameMapping) + sparkPartition, + spec, + serializableConf, + metricsConfig, + nameMapping, + service) .iterator(), Encoders.javaSerialization(DataFile.class)); @@ -635,7 +829,7 @@ public static void importSparkPartitions( Table targetTable, PartitionSpec spec, String stagingDir) { - importSparkPartitions(spark, partitions, targetTable, spec, stagingDir, false); + importSparkPartitions(spark, partitions, targetTable, spec, stagingDir, false, 1); } public static List filterPartitions( @@ -657,6 +851,12 @@ private static void deleteManifests(FileIO io, List manifests) { .run(item -> io.deleteFile(item.path())); } + public static Dataset loadTable(SparkSession spark, Table table, long snapshotId) { + SparkTable sparkTable = new SparkTable(table, snapshotId, false); + DataSourceV2Relation relation = createRelation(sparkTable, ImmutableMap.of()); + return Dataset.ofRows(spark, relation); + } + public static Dataset loadMetadataTable( SparkSession spark, Table table, MetadataTableType type) { return loadMetadataTable(spark, table, type, ImmutableMap.of()); @@ -664,11 +864,16 @@ public static Dataset loadMetadataTable( public static Dataset loadMetadataTable( SparkSession spark, Table table, MetadataTableType type, Map extraOptions) { - SparkTable metadataTable = - new SparkTable(MetadataTableUtils.createMetadataTableInstance(table, type), false); + Table metadataTable = MetadataTableUtils.createMetadataTableInstance(table, type); + SparkTable sparkMetadataTable = new SparkTable(metadataTable, false); + DataSourceV2Relation relation = createRelation(sparkMetadataTable, extraOptions); + return Dataset.ofRows(spark, relation); + } + + private static DataSourceV2Relation createRelation( + SparkTable sparkTable, Map extraOptions) { CaseInsensitiveStringMap options = new CaseInsensitiveStringMap(extraOptions); - return Dataset.ofRows( - spark, DataSourceV2Relation.create(metadataTable, Some.empty(), Some.empty(), options)); + return DataSourceV2Relation.create(sparkTable, Option.empty(), Option.empty(), options); } /** diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/ComputeTableStatsSparkAction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/ComputeTableStatsSparkAction.java new file mode 100644 index 000000000000..a508021c1040 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/ComputeTableStatsSparkAction.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.actions; + +import java.io.IOException; +import java.util.List; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.iceberg.GenericBlobMetadata; +import org.apache.iceberg.GenericStatisticsFile; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.IcebergBuild; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StatisticsFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.actions.ComputeTableStats; +import org.apache.iceberg.actions.ImmutableComputeTableStats; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.puffin.Blob; +import org.apache.iceberg.puffin.Puffin; +import org.apache.iceberg.puffin.PuffinWriter; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.spark.JobGroupInfo; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Computes the statistics of the given columns and stores it as Puffin files. */ +public class ComputeTableStatsSparkAction extends BaseSparkAction + implements ComputeTableStats { + + private static final Logger LOG = LoggerFactory.getLogger(ComputeTableStatsSparkAction.class); + private static final Result EMPTY_RESULT = ImmutableComputeTableStats.Result.builder().build(); + + private final Table table; + private List columns; + private Snapshot snapshot; + + ComputeTableStatsSparkAction(SparkSession spark, Table table) { + super(spark); + this.table = table; + this.snapshot = table.currentSnapshot(); + } + + @Override + protected ComputeTableStatsSparkAction self() { + return this; + } + + @Override + public ComputeTableStats columns(String... newColumns) { + Preconditions.checkArgument( + newColumns != null && newColumns.length > 0, "Columns cannot be null/empty"); + this.columns = ImmutableList.copyOf(ImmutableSet.copyOf(newColumns)); + return this; + } + + @Override + public ComputeTableStats snapshot(long newSnapshotId) { + Snapshot newSnapshot = table.snapshot(newSnapshotId); + Preconditions.checkArgument(newSnapshot != null, "Snapshot not found: %s", newSnapshotId); + this.snapshot = newSnapshot; + return this; + } + + @Override + public Result execute() { + if (snapshot == null) { + LOG.info("No snapshot to compute stats for table {}", table.name()); + return EMPTY_RESULT; + } + validateColumns(); + JobGroupInfo info = newJobGroupInfo("COMPUTE-TABLE-STATS", jobDesc()); + return withJobGroupInfo(info, this::doExecute); + } + + private Result doExecute() { + LOG.info( + "Computing stats for columns {} in {} (snapshot {})", + columns(), + table.name(), + snapshotId()); + List blobs = generateNDVBlobs(); + StatisticsFile statisticsFile = writeStatsFile(blobs); + table.updateStatistics().setStatistics(snapshotId(), statisticsFile).commit(); + return ImmutableComputeTableStats.Result.builder().statisticsFile(statisticsFile).build(); + } + + private StatisticsFile writeStatsFile(List blobs) { + LOG.info("Writing stats for table {} for snapshot {}", table.name(), snapshotId()); + OutputFile outputFile = table.io().newOutputFile(outputPath()); + try (PuffinWriter writer = Puffin.write(outputFile).createdBy(appIdentifier()).build()) { + blobs.forEach(writer::add); + writer.finish(); + return new GenericStatisticsFile( + snapshotId(), + outputFile.location(), + writer.fileSize(), + writer.footerSize(), + GenericBlobMetadata.from(writer.writtenBlobsMetadata())); + } catch (IOException e) { + throw new RuntimeIOException(e); + } + } + + private List generateNDVBlobs() { + return NDVSketchUtil.generateBlobs(spark(), table, snapshot, columns()); + } + + private List columns() { + if (columns == null) { + Schema schema = table.schemas().get(snapshot.schemaId()); + this.columns = + schema.columns().stream() + .filter(nestedField -> nestedField.type().isPrimitiveType()) + .map(Types.NestedField::name) + .collect(Collectors.toList()); + } + return columns; + } + + private void validateColumns() { + Schema schema = table.schemas().get(snapshot.schemaId()); + Preconditions.checkArgument(!columns().isEmpty(), "No columns found to compute stats"); + for (String columnName : columns()) { + Types.NestedField field = schema.findField(columnName); + Preconditions.checkArgument(field != null, "Can't find column %s in %s", columnName, schema); + Preconditions.checkArgument( + field.type().isPrimitiveType(), + "Can't compute stats on non-primitive type column: %s (%s)", + columnName, + field.type()); + } + } + + private String appIdentifier() { + String icebergVersion = IcebergBuild.fullVersion(); + String sparkVersion = spark().version(); + return String.format("Iceberg %s Spark %s", icebergVersion, sparkVersion); + } + + private long snapshotId() { + return snapshot.snapshotId(); + } + + private String jobDesc() { + return String.format( + "Computing table stats for %s (snapshot_id=%s, columns=%s)", + table.name(), snapshotId(), columns()); + } + + private String outputPath() { + TableOperations operations = ((HasTableOperations) table).operations(); + String fileName = String.format("%s-%s.stats", snapshotId(), UUID.randomUUID()); + return operations.metadataFileLocation(fileName); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateTableSparkAction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateTableSparkAction.java index 5f3cdd3f035c..bdffeb465405 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateTableSparkAction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateTableSparkAction.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.actions; import java.util.Map; +import java.util.concurrent.ExecutorService; import org.apache.iceberg.Snapshot; import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.Table; @@ -59,6 +60,7 @@ public class MigrateTableSparkAction extends BaseTableCreationSparkAction generateBlobs( + SparkSession spark, Table table, Snapshot snapshot, List columns) { + Row sketches = computeNDVSketches(spark, table, snapshot, columns); + Schema schema = table.schemas().get(snapshot.schemaId()); + List blobs = Lists.newArrayList(); + for (int i = 0; i < columns.size(); i++) { + Types.NestedField field = schema.findField(columns.get(i)); + Sketch sketch = CompactSketch.wrap(Memory.wrap((byte[]) sketches.get(i))); + blobs.add(toBlob(field, sketch, snapshot)); + } + return blobs; + } + + private static Blob toBlob(Types.NestedField field, Sketch sketch, Snapshot snapshot) { + return new Blob( + StandardBlobTypes.APACHE_DATASKETCHES_THETA_V1, + ImmutableList.of(field.fieldId()), + snapshot.snapshotId(), + snapshot.sequenceNumber(), + ByteBuffer.wrap(sketch.toByteArray()), + PuffinCompressionCodec.ZSTD, + ImmutableMap.of( + APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY, + String.valueOf((long) sketch.getEstimate()))); + } + + private static Row computeNDVSketches( + SparkSession spark, Table table, Snapshot snapshot, List colNames) { + Dataset inputDF = SparkTableUtil.loadTable(spark, table, snapshot.snapshotId()); + return inputDF.select(toAggColumns(colNames)).first(); + } + + private static Column[] toAggColumns(List colNames) { + return colNames.stream().map(NDVSketchUtil::toAggColumn).toArray(Column[]::new); + } + + private static Column toAggColumn(String colName) { + ThetaSketchAgg agg = new ThetaSketchAgg(colName); + return new Column(agg.toAggregateExpression()); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java new file mode 100644 index 000000000000..b9dc46f5e1bc --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java @@ -0,0 +1,171 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.actions; + +import static org.apache.spark.sql.functions.col; +import static org.apache.spark.sql.functions.min; + +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.Partitioning; +import org.apache.iceberg.RewriteFiles; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.ImmutableRemoveDanglingDeleteFiles; +import org.apache.iceberg.actions.RemoveDanglingDeleteFiles; +import org.apache.iceberg.spark.JobGroupInfo; +import org.apache.iceberg.spark.SparkDeleteFile; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.types.StructType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An action that removes dangling delete files from the current snapshot. A delete file is dangling + * if its deletes no longer applies to any live data files. + * + *

The following dangling delete files are removed: + * + *

    + *
  • Position delete files with a data sequence number less than that of any data file in the + * same partition + *
  • Equality delete files with a data sequence number less than or equal to that of any data + * file in the same partition + *
+ */ +class RemoveDanglingDeletesSparkAction + extends BaseSnapshotUpdateSparkAction + implements RemoveDanglingDeleteFiles { + private static final Logger LOG = LoggerFactory.getLogger(RemoveDanglingDeletesSparkAction.class); + private final Table table; + + protected RemoveDanglingDeletesSparkAction(SparkSession spark, Table table) { + super(spark); + this.table = table; + } + + @Override + protected RemoveDanglingDeletesSparkAction self() { + return this; + } + + public Result execute() { + if (table.specs().size() == 1 && table.spec().isUnpartitioned()) { + // ManifestFilterManager already performs this table-wide delete on each commit + return ImmutableRemoveDanglingDeleteFiles.Result.builder() + .removedDeleteFiles(Collections.emptyList()) + .build(); + } + String desc = String.format("Removing dangling delete files in %s", table.name()); + JobGroupInfo info = newJobGroupInfo("REMOVE-DELETES", desc); + return withJobGroupInfo(info, this::doExecute); + } + + Result doExecute() { + RewriteFiles rewriteFiles = table.newRewrite(); + List danglingDeletes = findDanglingDeletes(); + for (DeleteFile deleteFile : danglingDeletes) { + LOG.debug("Removing dangling delete file {}", deleteFile.path()); + rewriteFiles.deleteFile(deleteFile); + } + if (!danglingDeletes.isEmpty()) { + commit(rewriteFiles); + } + return ImmutableRemoveDanglingDeleteFiles.Result.builder() + .removedDeleteFiles(danglingDeletes) + .build(); + } + + /** + * Dangling delete files can be identified with following steps + * + *
    + *
  1. Group data files by partition keys and find the minimum data sequence number in each + * group. + *
  2. Left outer join delete files with partition-grouped data files on partition keys. + *
  3. Find dangling deletes by comparing each delete file's sequence number to its partition's + * minimum data sequence number. + *
  4. Collect results row to driver and use {@link SparkDeleteFile SparkDeleteFile} to wrap + * rows to valid delete files + *
+ */ + private List findDanglingDeletes() { + Dataset minSequenceNumberByPartition = + loadMetadataTable(table, MetadataTableType.ENTRIES) + // find live data files + .filter("data_file.content == 0 AND status < 2") + .selectExpr( + "data_file.partition as partition", + "data_file.spec_id as spec_id", + "sequence_number") + .groupBy("partition", "spec_id") + .agg(min("sequence_number")) + .toDF("grouped_partition", "grouped_spec_id", "min_data_sequence_number"); + Dataset deleteEntries = + loadMetadataTable(table, MetadataTableType.ENTRIES) + // find live delete files + .filter("data_file.content != 0 AND status < 2"); + Column joinOnPartition = + deleteEntries + .col("data_file.spec_id") + .equalTo(minSequenceNumberByPartition.col("grouped_spec_id")) + .and( + deleteEntries + .col("data_file.partition") + .equalTo(minSequenceNumberByPartition.col("grouped_partition"))); + Column filterOnDanglingDeletes = + col("min_data_sequence_number") + // delete fies without any data files in partition + .isNull() + // position delete files without any applicable data files in partition + .or( + col("data_file.content") + .equalTo("1") + .and(col("sequence_number").$less(col("min_data_sequence_number")))) + // equality delete files without any applicable data files in the partition + .or( + col("data_file.content") + .equalTo("2") + .and(col("sequence_number").$less$eq(col("min_data_sequence_number")))); + Dataset danglingDeletes = + deleteEntries + .join(minSequenceNumberByPartition, joinOnPartition, "left") + .filter(filterOnDanglingDeletes) + .select("data_file.*"); + return danglingDeletes.collectAsList().stream() + // map on driver because SparkDeleteFile is not serializable + .map(row -> deleteFileWrapper(danglingDeletes.schema(), row)) + .collect(Collectors.toList()); + } + + private DeleteFile deleteFileWrapper(StructType sparkFileType, Row row) { + int specId = row.getInt(row.fieldIndex("spec_id")); + Types.StructType combinedFileType = DataFile.getType(Partitioning.partitionType(table)); + // Set correct spec id + Types.StructType projection = DataFile.getType(table.specs().get(specId).partitionType()); + return new SparkDeleteFile(combinedFileType, projection, sparkFileType).wrap(row); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java index a4c6642a3edf..0b2bbb3dfc39 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java @@ -40,6 +40,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.actions.FileRewriter; import org.apache.iceberg.actions.ImmutableRewriteDataFiles; +import org.apache.iceberg.actions.ImmutableRewriteDataFiles.Result.Builder; import org.apache.iceberg.actions.RewriteDataFiles; import org.apache.iceberg.actions.RewriteDataFilesCommitManager; import org.apache.iceberg.actions.RewriteFileGroup; @@ -53,6 +54,7 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Queues; @@ -82,7 +84,8 @@ public class RewriteDataFilesSparkAction TARGET_FILE_SIZE_BYTES, USE_STARTING_SEQUENCE_NUMBER, REWRITE_JOB_ORDER, - OUTPUT_SPEC_ID); + OUTPUT_SPEC_ID, + REMOVE_DANGLING_DELETES); private static final RewriteDataFilesSparkAction.Result EMPTY_RESULT = ImmutableRewriteDataFiles.Result.builder().rewriteResults(ImmutableList.of()).build(); @@ -93,6 +96,7 @@ public class RewriteDataFilesSparkAction private int maxConcurrentFileGroupRewrites; private int maxCommits; private boolean partialProgressEnabled; + private boolean removeDanglingDeletes; private boolean useStartingSequenceNumber; private RewriteJobOrder rewriteJobOrder; private FileRewriter rewriter = null; @@ -173,11 +177,17 @@ public RewriteDataFiles.Result execute() { Stream groupStream = toGroupStream(ctx, fileGroupsByPartition); - if (partialProgressEnabled) { - return doExecuteWithPartialProgress(ctx, groupStream, commitManager(startingSnapshotId)); - } else { - return doExecute(ctx, groupStream, commitManager(startingSnapshotId)); + Builder resultBuilder = + partialProgressEnabled + ? doExecuteWithPartialProgress(ctx, groupStream, commitManager(startingSnapshotId)) + : doExecute(ctx, groupStream, commitManager(startingSnapshotId)); + if (removeDanglingDeletes) { + RemoveDanglingDeletesSparkAction action = + new RemoveDanglingDeletesSparkAction(spark(), table); + int removedCount = Iterables.size(action.execute().removedDeleteFiles()); + resultBuilder.removedDeleteFilesCount(removedCount); } + return resultBuilder.build(); } StructLikeMap>> planFileGroups(long startingSnapshotId) { @@ -261,7 +271,7 @@ RewriteDataFilesCommitManager commitManager(long startingSnapshotId) { return new RewriteDataFilesCommitManager(table, startingSnapshotId, useStartingSequenceNumber); } - private Result doExecute( + private Builder doExecute( RewriteExecutionContext ctx, Stream groupStream, RewriteDataFilesCommitManager commitManager) { @@ -323,10 +333,10 @@ private Result doExecute( List rewriteResults = rewrittenGroups.stream().map(RewriteFileGroup::asResult).collect(Collectors.toList()); - return ImmutableRewriteDataFiles.Result.builder().rewriteResults(rewriteResults).build(); + return ImmutableRewriteDataFiles.Result.builder().rewriteResults(rewriteResults); } - private Result doExecuteWithPartialProgress( + private Builder doExecuteWithPartialProgress( RewriteExecutionContext ctx, Stream groupStream, RewriteDataFilesCommitManager commitManager) { @@ -372,8 +382,7 @@ private Result doExecuteWithPartialProgress( commitResults.stream().map(RewriteFileGroup::asResult).collect(Collectors.toList()); return ImmutableRewriteDataFiles.Result.builder() .rewriteResults(rewriteResults) - .rewriteFailures(rewriteFailures) - .build(); + .rewriteFailures(rewriteFailures); } Stream toGroupStream( @@ -435,6 +444,10 @@ void validateAndInitOptions() { PropertyUtil.propertyAsBoolean( options(), USE_STARTING_SEQUENCE_NUMBER, USE_STARTING_SEQUENCE_NUMBER_DEFAULT); + removeDanglingDeletes = + PropertyUtil.propertyAsBoolean( + options(), REMOVE_DANGLING_DELETES, REMOVE_DANGLING_DELETES_DEFAULT); + rewriteJobOrder = RewriteJobOrder.fromName( PropertyUtil.propertyAsString(options(), REWRITE_JOB_ORDER, REWRITE_JOB_ORDER_DEFAULT)); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SnapshotTableSparkAction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SnapshotTableSparkAction.java index 8e59c13543f8..5f7f408cb099 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SnapshotTableSparkAction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SnapshotTableSparkAction.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.actions; import java.util.Map; +import java.util.concurrent.ExecutorService; import org.apache.iceberg.Snapshot; import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.Table; @@ -54,6 +55,7 @@ public class SnapshotTableSparkAction extends BaseTableCreationSparkAction partitionFilter, - boolean checkDuplicateFiles) { + boolean checkDuplicateFiles, + int parallelism) { return modifyIcebergTable( destIdent, table -> { @@ -153,9 +164,16 @@ private InternalRow[] importToIceberg( Path sourcePath = new Path(sourceIdent.name()); String format = sourceIdent.namespace()[0]; importFileTable( - table, sourcePath, format, partitionFilter, checkDuplicateFiles, table.spec()); + table, + sourcePath, + format, + partitionFilter, + checkDuplicateFiles, + table.spec(), + parallelism); } else { - importCatalogTable(table, sourceIdent, partitionFilter, checkDuplicateFiles); + importCatalogTable( + table, sourceIdent, partitionFilter, checkDuplicateFiles, parallelism); } Snapshot snapshot = table.currentSnapshot(); @@ -178,7 +196,8 @@ private void importFileTable( String format, Map partitionFilter, boolean checkDuplicateFiles, - PartitionSpec spec) { + PartitionSpec spec, + int parallelism) { // List Partitions via Spark InMemory file search interface List partitions = Spark3Util.getPartitions(spark(), tableLocation, format, partitionFilter, spec); @@ -193,11 +212,11 @@ private void importFileTable( // Build a Global Partition for the source SparkPartition partition = new SparkPartition(Collections.emptyMap(), tableLocation.toString(), format); - importPartitions(table, ImmutableList.of(partition), checkDuplicateFiles); + importPartitions(table, ImmutableList.of(partition), checkDuplicateFiles, parallelism); } else { Preconditions.checkArgument( !partitions.isEmpty(), "Cannot find any matching partitions in table %s", table.name()); - importPartitions(table, partitions, checkDuplicateFiles); + importPartitions(table, partitions, checkDuplicateFiles, parallelism); } } @@ -205,7 +224,8 @@ private void importCatalogTable( Table table, Identifier sourceIdent, Map partitionFilter, - boolean checkDuplicateFiles) { + boolean checkDuplicateFiles, + int parallelism) { String stagingLocation = getMetadataLocation(table); TableIdentifier sourceTableIdentifier = Spark3Util.toV1TableIdentifier(sourceIdent); SparkTableUtil.importSparkTable( @@ -214,14 +234,24 @@ private void importCatalogTable( table, stagingLocation, partitionFilter, - checkDuplicateFiles); + checkDuplicateFiles, + parallelism); } private void importPartitions( - Table table, List partitions, boolean checkDuplicateFiles) { + Table table, + List partitions, + boolean checkDuplicateFiles, + int parallelism) { String stagingLocation = getMetadataLocation(table); SparkTableUtil.importSparkPartitions( - spark(), partitions, table, table.spec(), stagingLocation, checkDuplicateFiles); + spark(), + partitions, + table, + table.spec(), + stagingLocation, + checkDuplicateFiles, + parallelism); } private String getMetadataLocation(Table table) { diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/MigrateTableProcedure.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/MigrateTableProcedure.java index 37b1e3bf0195..a0bd04dd997e 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/MigrateTableProcedure.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/MigrateTableProcedure.java @@ -40,7 +40,8 @@ class MigrateTableProcedure extends BaseProcedure { ProcedureParameter.required("table", DataTypes.StringType), ProcedureParameter.optional("properties", STRING_MAP), ProcedureParameter.optional("drop_backup", DataTypes.BooleanType), - ProcedureParameter.optional("backup_table_name", DataTypes.StringType) + ProcedureParameter.optional("backup_table_name", DataTypes.StringType), + ProcedureParameter.optional("parallelism", DataTypes.IntegerType) }; private static final StructType OUTPUT_TYPE = @@ -105,6 +106,13 @@ public InternalRow[] call(InternalRow args) { migrateTableSparkAction = migrateTableSparkAction.backupTableName(backupTableName); } + if (!args.isNullAt(4)) { + int parallelism = args.getInt(4); + Preconditions.checkArgument(parallelism > 0, "Parallelism should be larger than 0"); + migrateTableSparkAction = + migrateTableSparkAction.executeWith(executorService(parallelism, "table-migration")); + } + MigrateTable.Result result = migrateTableSparkAction.execute(); return new InternalRow[] {newInternalRow(result.migratedDataFilesCount())}; } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/ProcedureInput.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/ProcedureInput.java index 42e4d8ba0603..0be4b38de79c 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/ProcedureInput.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/ProcedureInput.java @@ -68,6 +68,18 @@ public Boolean asBoolean(ProcedureParameter param, Boolean defaultValue) { return args.isNullAt(ordinal) ? defaultValue : (Boolean) args.getBoolean(ordinal); } + public Integer asInt(ProcedureParameter param) { + Integer value = asInt(param, null); + Preconditions.checkArgument(value != null, "Parameter '%s' is not set", param.name()); + return value; + } + + public Integer asInt(ProcedureParameter param, Integer defaultValue) { + validateParamType(param, DataTypes.IntegerType); + int ordinal = ordinal(param); + return args.isNullAt(ordinal) ? defaultValue : (Integer) args.getInt(ordinal); + } + public long asLong(ProcedureParameter param) { Long value = asLong(param, null); Preconditions.checkArgument(value != null, "Parameter '%s' is not set", param.name()); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/SnapshotTableProcedure.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/SnapshotTableProcedure.java index 7a015a51e8ed..f709f64ebf62 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/SnapshotTableProcedure.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/SnapshotTableProcedure.java @@ -38,7 +38,8 @@ class SnapshotTableProcedure extends BaseProcedure { ProcedureParameter.required("source_table", DataTypes.StringType), ProcedureParameter.required("table", DataTypes.StringType), ProcedureParameter.optional("location", DataTypes.StringType), - ProcedureParameter.optional("properties", STRING_MAP) + ProcedureParameter.optional("properties", STRING_MAP), + ProcedureParameter.optional("parallelism", DataTypes.IntegerType) }; private static final StructType OUTPUT_TYPE = @@ -102,6 +103,12 @@ public InternalRow[] call(InternalRow args) { action.tableLocation(snapshotLocation); } + if (!args.isNullAt(4)) { + int parallelism = args.getInt(4); + Preconditions.checkArgument(parallelism > 0, "Parallelism should be larger than 0"); + action = action.executeWith(executorService(parallelism, "table-snapshot")); + } + SnapshotTable.Result result = action.tableProperties(properties).execute(); return new InternalRow[] {newInternalRow(result.importedDataFilesCount())}; } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java index 9df12fc060ae..c822ed743f85 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java @@ -49,6 +49,10 @@ import org.apache.spark.sql.types.StructType; import org.apache.spark.unsafe.types.UTF8String; +/** + * @deprecated since 1.7.0, will be removed in 1.8.0; use {@link SparkFileWriterFactory} instead. + */ +@Deprecated class SparkAppenderFactory implements FileAppenderFactory { private final Map properties; private final Schema writeSchema; diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java index 9dc214a755d3..d511fefd8ae0 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java @@ -49,7 +49,6 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.metrics.InMemoryMetricsReporter; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.Spark3Util; @@ -239,8 +238,7 @@ public boolean pushAggregation(Aggregation aggregation) { buildIcebergBatchScan(true /* include Column Stats */, schemaWithMetadataColumns()); try (CloseableIterable fileScanTasks = scan.planFiles()) { - List tasks = ImmutableList.copyOf(fileScanTasks); - for (FileScanTask task : tasks) { + for (FileScanTask task : fileScanTasks) { if (!task.deletes().isEmpty()) { LOG.info("Skipping aggregate pushdown: detected row level deletes"); return false; diff --git a/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala b/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala new file mode 100644 index 000000000000..cca16960f434 --- /dev/null +++ b/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.stats + +import java.nio.ByteBuffer +import org.apache.datasketches.common.Family +import org.apache.datasketches.memory.Memory +import org.apache.datasketches.theta.CompactSketch +import org.apache.datasketches.theta.SetOperationBuilder +import org.apache.datasketches.theta.Sketch +import org.apache.datasketches.theta.UpdateSketch +import org.apache.iceberg.spark.SparkSchemaUtil +import org.apache.iceberg.types.Conversions +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.expressions.aggregate.ImperativeAggregate +import org.apache.spark.sql.catalyst.expressions.aggregate.TypedImperativeAggregate +import org.apache.spark.sql.catalyst.trees.UnaryLike +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.types.BinaryType +import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.types.Decimal +import org.apache.spark.unsafe.types.UTF8String + +/** + * ThetaSketchAgg generates Alpha family sketch with default seed. + * The values fed to the sketch are converted to bytes using Iceberg's single value serialization. + * The result returned is an array of bytes of Compact Theta sketch of Datasketches library, + * which should be deserialized to Compact sketch before using. + * + * See [[https://iceberg.apache.org/puffin-spec/]] for more information. + * + */ +case class ThetaSketchAgg( + child: Expression, + mutableAggBufferOffset: Int = 0, + inputAggBufferOffset: Int = 0) extends TypedImperativeAggregate[Sketch] with UnaryLike[Expression] { + + private lazy val icebergType = SparkSchemaUtil.convert(child.dataType) + + def this(colName: String) = { + this(col(colName).expr, 0, 0) + } + + override def dataType: DataType = BinaryType + + override def nullable: Boolean = false + + override def createAggregationBuffer(): Sketch = { + UpdateSketch.builder.setFamily(Family.ALPHA).build() + } + + override def update(buffer: Sketch, input: InternalRow): Sketch = { + val value = child.eval(input) + if (value != null) { + val icebergValue = toIcebergValue(value) + val byteBuffer = Conversions.toByteBuffer(icebergType, icebergValue) + buffer.asInstanceOf[UpdateSketch].update(byteBuffer) + } + buffer + } + + private def toIcebergValue(value: Any): Any = { + value match { + case s: UTF8String => s.toString + case d: Decimal => d.toJavaBigDecimal + case b: Array[Byte] => ByteBuffer.wrap(b) + case _ => value + } + } + + override def merge(buffer: Sketch, input: Sketch): Sketch = { + new SetOperationBuilder().buildUnion.union(buffer, input) + } + + override def eval(buffer: Sketch): Any = { + toBytes(buffer) + } + + override def serialize(buffer: Sketch): Array[Byte] = { + toBytes(buffer) + } + + override def deserialize(storageFormat: Array[Byte]): Sketch = { + CompactSketch.wrap(Memory.wrap(storageFormat)) + } + + override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate = { + copy(mutableAggBufferOffset = newMutableAggBufferOffset) + } + + override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate = { + copy(inputAggBufferOffset = newInputAggBufferOffset) + } + + override protected def withNewChildInternal(newChild: Expression): Expression = { + copy(child = newChild) + } + + private def toBytes(sketch: Sketch): Array[Byte] = { + val compactSketch = sketch.compact() + compactSketch.toByteArray + } +} diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputeTableStatsAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputeTableStatsAction.java new file mode 100644 index 000000000000..88805a070cb1 --- /dev/null +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputeTableStatsAction.java @@ -0,0 +1,429 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.actions; + +import static org.apache.iceberg.spark.actions.NDVSketchUtil.APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatCode; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.BlobMetadata; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.Files; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StatisticsFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.ComputeTableStats; +import org.apache.iceberg.data.FileHelpers; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.spark.source.SimpleRecord; +import org.apache.iceberg.types.Types; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.parser.ParseException; +import org.apache.spark.sql.types.StructType; +import org.junit.After; +import org.junit.Test; + +public class TestComputeTableStatsAction extends SparkCatalogTestBase { + + private static final Types.StructType LEAF_STRUCT_TYPE = + Types.StructType.of( + optional(1, "leafLongCol", Types.LongType.get()), + optional(2, "leafDoubleCol", Types.DoubleType.get())); + + private static final Types.StructType NESTED_STRUCT_TYPE = + Types.StructType.of(required(3, "leafStructCol", LEAF_STRUCT_TYPE)); + + private static final Schema NESTED_SCHEMA = + new Schema(required(4, "nestedStructCol", NESTED_STRUCT_TYPE)); + + private static final Schema SCHEMA_WITH_NESTED_COLUMN = + new Schema( + required(4, "nestedStructCol", NESTED_STRUCT_TYPE), + required(5, "stringCol", Types.StringType.get())); + + public TestComputeTableStatsAction( + String catalogName, String implementation, Map config) { + super(catalogName, implementation, config); + } + + @Test + public void testLoadingTableDirectly() { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + sql("INSERT into %s values(1, 'abcd')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result results = actions.computeTableStats(table).execute(); + StatisticsFile statisticsFile = results.statisticsFile(); + assertThat(statisticsFile.fileSizeInBytes()).isNotEqualTo(0); + assertThat(statisticsFile.blobMetadata().size()).isEqualTo(2); + } + + @Test + public void testComputeTableStatsAction() throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + + // To create multiple splits on the mapper + table + .updateProperties() + .set("read.split.target-size", "100") + .set("write.parquet.row-group-size-bytes", "100") + .commit(); + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "c"), + new SimpleRecord(4, "d")); + spark.createDataset(records, Encoders.bean(SimpleRecord.class)).writeTo(tableName).append(); + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result results = + actions.computeTableStats(table).columns("id", "data").execute(); + assertThat(results).isNotNull(); + + List statisticsFiles = table.statisticsFiles(); + assertThat(statisticsFiles.size()).isEqualTo(1); + + StatisticsFile statisticsFile = statisticsFiles.get(0); + assertThat(statisticsFile.fileSizeInBytes()).isNotEqualTo(0); + assertThat(statisticsFile.blobMetadata().size()).isEqualTo(2); + + BlobMetadata blobMetadata = statisticsFile.blobMetadata().get(0); + assertThat(blobMetadata.properties().get(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY)) + .isEqualTo(String.valueOf(4)); + } + + @Test + public void testComputeTableStatsActionWithoutExplicitColumns() + throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "c"), + new SimpleRecord(4, "d")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result results = actions.computeTableStats(table).execute(); + assertThat(results).isNotNull(); + + assertThat(table.statisticsFiles().size()).isEqualTo(1); + StatisticsFile statisticsFile = table.statisticsFiles().get(0); + assertThat(statisticsFile.blobMetadata().size()).isEqualTo(2); + assertThat(statisticsFile.fileSizeInBytes()).isNotEqualTo(0); + assertThat( + Long.parseLong( + statisticsFile + .blobMetadata() + .get(0) + .properties() + .get(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY))) + .isEqualTo(4); + assertThat( + Long.parseLong( + statisticsFile + .blobMetadata() + .get(1) + .properties() + .get(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY))) + .isEqualTo(4); + } + + @Test + public void testComputeTableStatsForInvalidColumns() throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + // Append data to create snapshot + sql("INSERT into %s values(1, 'abcd')", tableName); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + assertThatThrownBy(() -> actions.computeTableStats(table).columns("id1").execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Can't find column id1 in table"); + } + + @Test + public void testComputeTableStatsWithNoSnapshots() throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result result = actions.computeTableStats(table).columns("id").execute(); + assertThat(result.statisticsFile()).isNull(); + } + + @Test + public void testComputeTableStatsWithNullValues() throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + List records = + Lists.newArrayList( + new SimpleRecord(1, null), + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "c"), + new SimpleRecord(4, "d")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result results = actions.computeTableStats(table).columns("data").execute(); + assertThat(results).isNotNull(); + + List statisticsFiles = table.statisticsFiles(); + assertThat(statisticsFiles.size()).isEqualTo(1); + + StatisticsFile statisticsFile = statisticsFiles.get(0); + assertThat(statisticsFile.fileSizeInBytes()).isNotEqualTo(0); + assertThat(statisticsFile.blobMetadata().size()).isEqualTo(1); + + BlobMetadata blobMetadata = statisticsFile.blobMetadata().get(0); + assertThat(blobMetadata.properties().get(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY)) + .isEqualTo(String.valueOf(4)); + } + + @Test + public void testComputeTableStatsWithSnapshotHavingDifferentSchemas() + throws NoSuchTableException, ParseException { + SparkActions actions = SparkActions.get(); + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + // Append data to create snapshot + sql("INSERT into %s values(1, 'abcd')", tableName); + long snapshotId1 = Spark3Util.loadIcebergTable(spark, tableName).currentSnapshot().snapshotId(); + // Snapshot id not specified + Table table = Spark3Util.loadIcebergTable(spark, tableName); + + assertThatCode(() -> actions.computeTableStats(table).columns("data").execute()) + .doesNotThrowAnyException(); + + sql("ALTER TABLE %s DROP COLUMN %s", tableName, "data"); + // Append data to create snapshot + sql("INSERT into %s values(1)", tableName); + table.refresh(); + long snapshotId2 = Spark3Util.loadIcebergTable(spark, tableName).currentSnapshot().snapshotId(); + + // Snapshot id specified + assertThatCode( + () -> actions.computeTableStats(table).snapshot(snapshotId1).columns("data").execute()) + .doesNotThrowAnyException(); + + assertThatThrownBy( + () -> actions.computeTableStats(table).snapshot(snapshotId2).columns("data").execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Can't find column data in table"); + } + + @Test + public void testComputeTableStatsWhenSnapshotIdNotSpecified() + throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + // Append data to create snapshot + sql("INSERT into %s values(1, 'abcd')", tableName); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result results = actions.computeTableStats(table).columns("data").execute(); + + assertThat(results).isNotNull(); + + List statisticsFiles = table.statisticsFiles(); + assertThat(statisticsFiles.size()).isEqualTo(1); + + StatisticsFile statisticsFile = statisticsFiles.get(0); + assertThat(statisticsFile.fileSizeInBytes()).isNotEqualTo(0); + assertThat(statisticsFile.blobMetadata().size()).isEqualTo(1); + + BlobMetadata blobMetadata = statisticsFile.blobMetadata().get(0); + assertThat(blobMetadata.properties().get(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY)) + .isEqualTo(String.valueOf(1)); + } + + @Test + public void testComputeTableStatsWithNestedSchema() + throws NoSuchTableException, ParseException, IOException { + List records = Lists.newArrayList(createNestedRecord()); + Table table = + validationCatalog.createTable( + tableIdent, + SCHEMA_WITH_NESTED_COLUMN, + PartitionSpec.unpartitioned(), + ImmutableMap.of()); + DataFile dataFile = + FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), records); + table.newAppend().appendFile(dataFile).commit(); + + Table tbl = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + actions.computeTableStats(tbl).execute(); + + tbl.refresh(); + List statisticsFiles = tbl.statisticsFiles(); + assertThat(statisticsFiles.size()).isEqualTo(1); + StatisticsFile statisticsFile = statisticsFiles.get(0); + assertThat(statisticsFile.fileSizeInBytes()).isNotEqualTo(0); + assertThat(statisticsFile.blobMetadata().size()).isEqualTo(1); + } + + @Test + public void testComputeTableStatsWithNoComputableColumns() throws IOException { + List records = Lists.newArrayList(createNestedRecord()); + Table table = + validationCatalog.createTable( + tableIdent, NESTED_SCHEMA, PartitionSpec.unpartitioned(), ImmutableMap.of()); + DataFile dataFile = + FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), records); + table.newAppend().appendFile(dataFile).commit(); + + table.refresh(); + SparkActions actions = SparkActions.get(); + assertThatThrownBy(() -> actions.computeTableStats(table).execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("No columns found to compute stats"); + } + + @Test + public void testComputeTableStatsOnByteColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("byte_col", "TINYINT"); + } + + @Test + public void testComputeTableStatsOnShortColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("short_col", "SMALLINT"); + } + + @Test + public void testComputeTableStatsOnIntColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("int_col", "INT"); + } + + @Test + public void testComputeTableStatsOnLongColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("long_col", "BIGINT"); + } + + @Test + public void testComputeTableStatsOnTimestampColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("timestamp_col", "TIMESTAMP"); + } + + @Test + public void testComputeTableStatsOnTimestampNtzColumn() + throws NoSuchTableException, ParseException { + testComputeTableStats("timestamp_col", "TIMESTAMP_NTZ"); + } + + @Test + public void testComputeTableStatsOnDateColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("date_col", "DATE"); + } + + @Test + public void testComputeTableStatsOnDecimalColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("decimal_col", "DECIMAL(20, 2)"); + } + + @Test + public void testComputeTableStatsOnBinaryColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("binary_col", "BINARY"); + } + + public void testComputeTableStats(String columnName, String type) + throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, %s %s) USING iceberg", tableName, columnName, type); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + + Dataset dataDF = randomDataDF(table.schema()); + append(tableName, dataDF); + + SparkActions actions = SparkActions.get(); + table.refresh(); + ComputeTableStats.Result results = + actions.computeTableStats(table).columns(columnName).execute(); + assertThat(results).isNotNull(); + + List statisticsFiles = table.statisticsFiles(); + assertThat(statisticsFiles.size()).isEqualTo(1); + + StatisticsFile statisticsFile = statisticsFiles.get(0); + assertThat(statisticsFile.fileSizeInBytes()).isNotEqualTo(0); + assertThat(statisticsFile.blobMetadata().size()).isEqualTo(1); + + BlobMetadata blobMetadata = statisticsFile.blobMetadata().get(0); + assertThat(blobMetadata.properties().get(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY)) + .isNotNull(); + } + + private GenericRecord createNestedRecord() { + GenericRecord record = GenericRecord.create(SCHEMA_WITH_NESTED_COLUMN); + GenericRecord nested = GenericRecord.create(NESTED_STRUCT_TYPE); + GenericRecord leaf = GenericRecord.create(LEAF_STRUCT_TYPE); + leaf.set(0, 0L); + leaf.set(1, 0.0); + nested.set(0, leaf); + record.set(0, nested); + record.set(1, "data"); + return record; + } + + private Dataset randomDataDF(Schema schema) { + Iterable rows = RandomData.generateSpark(schema, 10, 0); + JavaRDD rowRDD = sparkContext.parallelize(Lists.newArrayList(rows)); + StructType rowSparkType = SparkSchemaUtil.convert(schema); + return spark.internalCreateDataFrame(JavaRDD.toRDD(rowRDD), rowSparkType, false); + } + + private void append(String table, Dataset df) throws NoSuchTableException { + // fanout writes are enabled as write-time clustering is not supported without Spark extensions + df.coalesce(1).writeTo(table).option(SparkWriteOptions.FANOUT_ENABLED, "true").append(); + } + + @After + public void removeTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + } +} diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java index 028d495b894d..06f118be0c6f 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java @@ -183,6 +183,11 @@ public void before() { public void after() throws IOException { // Drop the hive table. spark.sql(String.format("DROP TABLE IF EXISTS %s", baseTableName)); + spark.sessionState().catalogManager().reset(); + spark.conf().unset("spark.sql.catalog.spark_catalog.type"); + spark.conf().unset("spark.sql.catalog.spark_catalog.default-namespace"); + spark.conf().unset("spark.sql.catalog.spark_catalog.parquet-enabled"); + spark.conf().unset("spark.sql.catalog.spark_catalog.cache-enabled"); } @Test @@ -728,6 +733,8 @@ public void testStructOfThreeLevelLists() throws Exception { @Test public void testTwoLevelList() throws IOException { + Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", true); String tableName = sourceName("testTwoLevelList"); @@ -811,6 +818,8 @@ public boolean accept(File dir, String name) { } private void threeLevelList(boolean useLegacyMode) throws Exception { + Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); String tableName = sourceName(String.format("threeLevelList_%s", useLegacyMode)); @@ -833,6 +842,8 @@ private void threeLevelList(boolean useLegacyMode) throws Exception { } private void threeLevelListWithNestedStruct(boolean useLegacyMode) throws Exception { + Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); String tableName = @@ -858,6 +869,8 @@ private void threeLevelListWithNestedStruct(boolean useLegacyMode) throws Except } private void threeLevelLists(boolean useLegacyMode) throws Exception { + Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); String tableName = sourceName(String.format("threeLevelLists_%s", useLegacyMode)); @@ -885,6 +898,8 @@ private void threeLevelLists(boolean useLegacyMode) throws Exception { } private void structOfThreeLevelLists(boolean useLegacyMode) throws Exception { + Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); String tableName = sourceName(String.format("structOfThreeLevelLists_%s", useLegacyMode)); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateTableAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateTableAction.java new file mode 100644 index 000000000000..7bed72b7cc2c --- /dev/null +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateTableAction.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.actions; + +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; + +public class TestMigrateTableAction extends SparkCatalogTestBase { + + public TestMigrateTableAction( + String catalogName, String implementation, Map config) { + super(catalogName, implementation, config); + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); + } + + @After + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS %s_BACKUP_", tableName); + } + + @Test + public void testMigrateWithParallelTasks() throws IOException { + String location = temp.newFolder().toURI().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + tableName, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); + + AtomicInteger migrationThreadsIndex = new AtomicInteger(0); + SparkActions.get() + .migrateTable(tableName) + .executeWith( + Executors.newFixedThreadPool( + 4, + runnable -> { + Thread thread = new Thread(runnable); + thread.setName("table-migration-" + migrationThreadsIndex.getAndIncrement()); + thread.setDaemon(true); + return thread; + })) + .execute(); + Assert.assertEquals(migrationThreadsIndex.get(), 2); + } +} diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java new file mode 100644 index 000000000000..8ac9be00ae90 --- /dev/null +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java @@ -0,0 +1,426 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.actions; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.actions.RemoveDanglingDeleteFiles; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.spark.SparkTestBase; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Encoders; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import scala.Tuple2; + +public class TestRemoveDanglingDeleteAction extends SparkTestBase { + private static final HadoopTables TABLES = new HadoopTables(new Configuration()); + private static final Schema SCHEMA = + new Schema( + optional(1, "c1", Types.StringType.get()), + optional(2, "c2", Types.StringType.get()), + optional(3, "c3", Types.StringType.get())); + private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build(); + static final DataFile FILE_A = + DataFiles.builder(SPEC) + .withPath("/path/to/data-a.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=a") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_A2 = + DataFiles.builder(SPEC) + .withPath("/path/to/data-a.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=a") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_B = + DataFiles.builder(SPEC) + .withPath("/path/to/data-b.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=b") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_B2 = + DataFiles.builder(SPEC) + .withPath("/path/to/data-b.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=b") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_C = + DataFiles.builder(SPEC) + .withPath("/path/to/data-c.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=c") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_C2 = + DataFiles.builder(SPEC) + .withPath("/path/to/data-c.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=c") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_D = + DataFiles.builder(SPEC) + .withPath("/path/to/data-d.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=d") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_D2 = + DataFiles.builder(SPEC) + .withPath("/path/to/data-d.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=d") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_A_POS_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("/path/to/data-a-pos-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=a") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_A2_POS_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("/path/to/data-a2-pos-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=a") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_A_EQ_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofEqualityDeletes() + .withPath("/path/to/data-a-eq-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=a") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_A2_EQ_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofEqualityDeletes() + .withPath("/path/to/data-a2-eq-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=a") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_B_POS_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("/path/to/data-b-pos-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=b") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_B2_POS_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("/path/to/data-b2-pos-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=b") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_B_EQ_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofEqualityDeletes() + .withPath("/path/to/data-b-eq-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=b") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_B2_EQ_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofEqualityDeletes() + .withPath("/path/to/data-b2-eq-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=b") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_UNPARTITIONED = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-unpartitioned.parquet") + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + static final DeleteFile FILE_UNPARTITIONED_POS_DELETE = + FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofEqualityDeletes() + .withPath("/path/to/data-unpartitioned-pos-deletes.parquet") + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + static final DeleteFile FILE_UNPARTITIONED_EQ_DELETE = + FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofEqualityDeletes() + .withPath("/path/to/data-unpartitioned-eq-deletes.parquet") + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + @Rule public TemporaryFolder temp = new TemporaryFolder(); + + private String tableLocation = null; + private Table table; + + @Before + public void before() throws Exception { + File tableDir = temp.newFolder(); + this.tableLocation = tableDir.toURI().toString(); + } + + @After + public void after() { + TABLES.dropTable(tableLocation); + } + + private void setupPartitionedTable() { + this.table = + TABLES.create( + SCHEMA, SPEC, ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"), tableLocation); + } + + private void setupUnpartitionedTable() { + this.table = + TABLES.create( + SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"), + tableLocation); + } + + @Test + public void testPartitionedDeletesWithLesserSeqNo() { + setupPartitionedTable(); + // Add Data Files + table.newAppend().appendFile(FILE_B).appendFile(FILE_C).appendFile(FILE_D).commit(); + // Add Delete Files + table + .newRowDelta() + .addDeletes(FILE_A_POS_DELETES) + .addDeletes(FILE_A2_POS_DELETES) + .addDeletes(FILE_B_POS_DELETES) + .addDeletes(FILE_B2_POS_DELETES) + .addDeletes(FILE_A_EQ_DELETES) + .addDeletes(FILE_A2_EQ_DELETES) + .addDeletes(FILE_B_EQ_DELETES) + .addDeletes(FILE_B2_EQ_DELETES) + .commit(); + // Add More Data Files + table + .newAppend() + .appendFile(FILE_A2) + .appendFile(FILE_B2) + .appendFile(FILE_C2) + .appendFile(FILE_D2) + .commit(); + List> actual = + spark + .read() + .format("iceberg") + .load(tableLocation + "#entries") + .select("sequence_number", "data_file.file_path") + .sort("sequence_number", "data_file.file_path") + .as(Encoders.tuple(Encoders.LONG(), Encoders.STRING())) + .collectAsList(); + List> expected = + ImmutableList.of( + Tuple2.apply(1L, FILE_B.path().toString()), + Tuple2.apply(1L, FILE_C.path().toString()), + Tuple2.apply(1L, FILE_D.path().toString()), + Tuple2.apply(2L, FILE_A_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A2_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A2_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B2_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B2_POS_DELETES.path().toString()), + Tuple2.apply(3L, FILE_A2.path().toString()), + Tuple2.apply(3L, FILE_B2.path().toString()), + Tuple2.apply(3L, FILE_C2.path().toString()), + Tuple2.apply(3L, FILE_D2.path().toString())); + assertThat(actual).isEqualTo(expected); + RemoveDanglingDeleteFiles.Result result = + SparkActions.get().removeDanglingDeleteFiles(table).execute(); + // All Delete files of the FILE A partition should be removed + // because there are no data files in partition with a lesser sequence number + Set removedDeleteFiles = + StreamSupport.stream(result.removedDeleteFiles().spliterator(), false) + .map(DeleteFile::path) + .collect(Collectors.toSet()); + assertThat(removedDeleteFiles) + .as("Expected 4 delete files removed") + .hasSize(4) + .containsExactlyInAnyOrder( + FILE_A_POS_DELETES.path(), + FILE_A2_POS_DELETES.path(), + FILE_A_EQ_DELETES.path(), + FILE_A2_EQ_DELETES.path()); + List> actualAfter = + spark + .read() + .format("iceberg") + .load(tableLocation + "#entries") + .filter("status < 2") // live files + .select("sequence_number", "data_file.file_path") + .sort("sequence_number", "data_file.file_path") + .as(Encoders.tuple(Encoders.LONG(), Encoders.STRING())) + .collectAsList(); + List> expectedAfter = + ImmutableList.of( + Tuple2.apply(1L, FILE_B.path().toString()), + Tuple2.apply(1L, FILE_C.path().toString()), + Tuple2.apply(1L, FILE_D.path().toString()), + Tuple2.apply(2L, FILE_B_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B2_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B2_POS_DELETES.path().toString()), + Tuple2.apply(3L, FILE_A2.path().toString()), + Tuple2.apply(3L, FILE_B2.path().toString()), + Tuple2.apply(3L, FILE_C2.path().toString()), + Tuple2.apply(3L, FILE_D2.path().toString())); + assertThat(actualAfter).isEqualTo(expectedAfter); + } + + @Test + public void testPartitionedDeletesWithEqSeqNo() { + setupPartitionedTable(); + // Add Data Files + table.newAppend().appendFile(FILE_A).appendFile(FILE_C).appendFile(FILE_D).commit(); + // Add Data Files with EQ and POS deletes + table + .newRowDelta() + .addRows(FILE_A2) + .addRows(FILE_B2) + .addRows(FILE_C2) + .addRows(FILE_D2) + .addDeletes(FILE_A_POS_DELETES) + .addDeletes(FILE_A2_POS_DELETES) + .addDeletes(FILE_A_EQ_DELETES) + .addDeletes(FILE_A2_EQ_DELETES) + .addDeletes(FILE_B_POS_DELETES) + .addDeletes(FILE_B2_POS_DELETES) + .addDeletes(FILE_B_EQ_DELETES) + .addDeletes(FILE_B2_EQ_DELETES) + .commit(); + List> actual = + spark + .read() + .format("iceberg") + .load(tableLocation + "#entries") + .select("sequence_number", "data_file.file_path") + .sort("sequence_number", "data_file.file_path") + .as(Encoders.tuple(Encoders.LONG(), Encoders.STRING())) + .collectAsList(); + List> expected = + ImmutableList.of( + Tuple2.apply(1L, FILE_A.path().toString()), + Tuple2.apply(1L, FILE_C.path().toString()), + Tuple2.apply(1L, FILE_D.path().toString()), + Tuple2.apply(2L, FILE_A_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A2.path().toString()), + Tuple2.apply(2L, FILE_A2_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A2_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B2.path().toString()), + Tuple2.apply(2L, FILE_B2_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B2_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_C2.path().toString()), + Tuple2.apply(2L, FILE_D2.path().toString())); + assertThat(actual).isEqualTo(expected); + RemoveDanglingDeleteFiles.Result result = + SparkActions.get().removeDanglingDeleteFiles(table).execute(); + // Eq Delete files of the FILE B partition should be removed + // because there are no data files in partition with a lesser sequence number + Set removedDeleteFiles = + StreamSupport.stream(result.removedDeleteFiles().spliterator(), false) + .map(DeleteFile::path) + .collect(Collectors.toSet()); + assertThat(removedDeleteFiles) + .as("Expected two delete files removed") + .hasSize(2) + .containsExactlyInAnyOrder(FILE_B_EQ_DELETES.path(), FILE_B2_EQ_DELETES.path()); + List> actualAfter = + spark + .read() + .format("iceberg") + .load(tableLocation + "#entries") + .filter("status < 2") // live files + .select("sequence_number", "data_file.file_path") + .sort("sequence_number", "data_file.file_path") + .as(Encoders.tuple(Encoders.LONG(), Encoders.STRING())) + .collectAsList(); + List> expectedAfter = + ImmutableList.of( + Tuple2.apply(1L, FILE_A.path().toString()), + Tuple2.apply(1L, FILE_C.path().toString()), + Tuple2.apply(1L, FILE_D.path().toString()), + Tuple2.apply(2L, FILE_A_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A2.path().toString()), + Tuple2.apply(2L, FILE_A2_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A2_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B2.path().toString()), + Tuple2.apply(2L, FILE_B2_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_C2.path().toString()), + Tuple2.apply(2L, FILE_D2.path().toString())); + assertThat(actualAfter).isEqualTo(expectedAfter); + } + + @Test + public void testUnpartitionedTable() { + setupUnpartitionedTable(); + table + .newRowDelta() + .addDeletes(FILE_UNPARTITIONED_POS_DELETE) + .addDeletes(FILE_UNPARTITIONED_EQ_DELETE) + .commit(); + table.newAppend().appendFile(FILE_UNPARTITIONED).commit(); + RemoveDanglingDeleteFiles.Result result = + SparkActions.get().removeDanglingDeleteFiles(table).execute(); + assertThat(result.removedDeleteFiles()).as("No-op for unpartitioned tables").isEmpty(); + } +} diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java index ba173d02498e..656b7358d1a5 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java @@ -24,6 +24,7 @@ import static org.apache.spark.sql.functions.current_date; import static org.apache.spark.sql.functions.date_add; import static org.apache.spark.sql.functions.expr; +import static org.apache.spark.sql.functions.min; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.mockito.ArgumentMatchers.any; @@ -55,6 +56,7 @@ import org.apache.iceberg.FileScanTask; import org.apache.iceberg.MetadataTableType; import org.apache.iceberg.PartitionData; +import org.apache.iceberg.PartitionKey; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.RewriteJobOrder; import org.apache.iceberg.RowDelta; @@ -71,7 +73,9 @@ import org.apache.iceberg.actions.SizeBasedDataRewriter; import org.apache.iceberg.actions.SizeBasedFileRewriter; import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; +import org.apache.iceberg.deletes.EqualityDeleteWriter; import org.apache.iceberg.deletes.PositionDelete; import org.apache.iceberg.deletes.PositionDeleteWriter; import org.apache.iceberg.encryption.EncryptedFiles; @@ -84,6 +88,7 @@ import org.apache.iceberg.hadoop.HadoopTables; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.OutputFileFactory; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -104,9 +109,11 @@ import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.NestedField; +import org.apache.iceberg.util.ArrayUtil; import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.StructLikeMap; import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; import org.apache.spark.sql.internal.SQLConf; import org.junit.Assert; @@ -129,6 +136,8 @@ public class TestRewriteDataFilesAction extends SparkTestBase { optional(2, "c2", Types.StringType.get()), optional(3, "c3", Types.StringType.get())); + private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build(); + @Rule public TemporaryFolder temp = new TemporaryFolder(); private final FileRewriteCoordinator coordinator = FileRewriteCoordinator.get(); @@ -330,6 +339,108 @@ public void testBinPackWithDeletes() { Assert.assertEquals("7 rows are removed", total - 7, actualRecords.size()); } + @Test + public void testRemoveDangledEqualityDeletesPartitionEvolution() { + Table table = + TABLES.create( + SCHEMA, + SPEC, + Collections.singletonMap(TableProperties.FORMAT_VERSION, "2"), + tableLocation); + // data seq = 1, write 4 files in 2 partitions + List records1 = + Lists.newArrayList( + new ThreeColumnRecord(1, null, "AAAA"), new ThreeColumnRecord(1, "BBBBBBBBBB", "BBBB")); + writeRecords(records1); + List records2 = + Lists.newArrayList( + new ThreeColumnRecord(0, "CCCCCCCCCC", "CCCC"), + new ThreeColumnRecord(0, "DDDDDDDDDD", "DDDD")); + writeRecords(records2); + table.refresh(); + shouldHaveFiles(table, 4); + // data seq = 2 & 3, write 2 equality deletes in both partitions + writeEqDeleteRecord(table, "c1", 1, "c3", "AAAA"); + writeEqDeleteRecord(table, "c1", 2, "c3", "CCCC"); + table.refresh(); + Set existingDeletes = TestHelpers.deleteFiles(table); + assertThat(existingDeletes) + .as("Only one equality delete c1=1 is used in query planning") + .hasSize(1); + // partition evolution + table.refresh(); + table.updateSpec().addField(Expressions.ref("c3")).commit(); + // data seq = 4, write 2 new data files in both partitions for evolved spec + List records3 = + Lists.newArrayList( + new ThreeColumnRecord(1, "A", "CCCC"), new ThreeColumnRecord(2, "D", "DDDD")); + writeRecords(records3); + List originalData = currentData(); + RewriteDataFiles.Result result = + basicRewrite(table) + .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .filter(Expressions.equal("c1", 1)) + .option(RewriteDataFiles.REMOVE_DANGLING_DELETES, "true") + .execute(); + existingDeletes = TestHelpers.deleteFiles(table); + assertThat(existingDeletes).as("Shall pruned dangling deletes after rewrite").hasSize(0); + assertThat(result) + .extracting( + Result::addedDataFilesCount, + Result::rewrittenDataFilesCount, + Result::removedDeleteFilesCount) + .as("Should compact 3 data files into 2 and remove both dangled equality delete file") + .containsExactly(2, 3, 2); + shouldHaveMinSequenceNumberInPartition(table, "data_file.partition.c1 == 1", 5); + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + shouldHaveSnapshots(table, 7); + shouldHaveFiles(table, 5); + } + + @Test + public void testRemoveDangledPositionDeletesPartitionEvolution() { + Table table = + TABLES.create( + SCHEMA, + SPEC, + Collections.singletonMap(TableProperties.FORMAT_VERSION, "2"), + tableLocation); + // data seq = 1, write 4 files in 2 partitions + writeRecords(2, 2, 2); + List dataFilesBefore = TestHelpers.dataFiles(table, null); + shouldHaveFiles(table, 4); + // data seq = 2, write 1 position deletes in c1=1 + table + .newRowDelta() + .addDeletes(writePosDeletesToFile(table, dataFilesBefore.get(3), 1).get(0)) + .commit(); + // partition evolution + table.updateSpec().addField(Expressions.ref("c3")).commit(); + // data seq = 3, write 1 new data files in c1=1 for evolved spec + writeRecords(1, 1, 1); + shouldHaveFiles(table, 5); + List expectedRecords = currentData(); + Result result = + actions() + .rewriteDataFiles(table) + .filter(Expressions.equal("c1", 1)) + .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(RewriteDataFiles.REMOVE_DANGLING_DELETES, "true") + .execute(); + assertThat(result) + .extracting( + Result::addedDataFilesCount, + Result::rewrittenDataFilesCount, + Result::removedDeleteFilesCount) + .as("Should rewrite 2 data files into 1 and remove 1 dangled position delete file") + .containsExactly(1, 2, 1); + shouldHaveMinSequenceNumberInPartition(table, "data_file.partition.c1 == 1", 3); + shouldHaveSnapshots(table, 5); + assertThat(table.currentSnapshot().summary().get("total-position-deletes")).isEqualTo("0"); + assertEquals("Rows must match", expectedRecords, currentData()); + } + @Test public void testBinPackWithDeleteAllData() { Map options = Maps.newHashMap(); @@ -1616,6 +1727,21 @@ protected void shouldHaveFiles(Table table, int numExpected) { Assert.assertEquals("Did not have the expected number of files", numExpected, numFiles); } + protected long shouldHaveMinSequenceNumberInPartition( + Table table, String partitionFilter, long expected) { + long actual = + SparkTableUtil.loadMetadataTable(spark, table, MetadataTableType.ENTRIES) + .filter("status != 2") + .filter(partitionFilter) + .select("sequence_number") + .agg(min("sequence_number")) + .as(Encoders.LONG()) + .collectAsList() + .get(0); + assertThat(actual).as("Did not have the expected min sequence number").isEqualTo(expected); + return actual; + } + protected void shouldHaveSnapshots(Table table, int expectedSnapshots) { table.refresh(); int actualSnapshots = Iterables.size(table.snapshots()); @@ -1812,6 +1938,11 @@ protected int averageFileSize(Table table) { .getAsDouble(); } + private void writeRecords(List records) { + Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class); + writeDF(df); + } + private void writeRecords(int files, int numRecords) { writeRecords(files, numRecords, 0); } @@ -1865,7 +1996,10 @@ private List writePosDeletes( table .io() .newOutputFile( - table.locationProvider().newDataLocation(UUID.randomUUID().toString())); + table + .locationProvider() + .newDataLocation( + FileFormat.PARQUET.addExtension(UUID.randomUUID().toString()))); EncryptedOutputFile encryptedOutputFile = EncryptedFiles.encryptedOutput(outputFile, EncryptionKeyMetadata.EMPTY); @@ -1891,6 +2025,63 @@ private List writePosDeletes( return results; } + private void writeEqDeleteRecord( + Table table, String partCol, Object partVal, String delCol, Object delVal) { + List equalityFieldIds = Lists.newArrayList(table.schema().findField(delCol).fieldId()); + Schema eqDeleteRowSchema = table.schema().select(delCol); + Record partitionRecord = + GenericRecord.create(table.schema().select(partCol)) + .copy(ImmutableMap.of(partCol, partVal)); + Record record = GenericRecord.create(eqDeleteRowSchema).copy(ImmutableMap.of(delCol, delVal)); + writeEqDeleteRecord(table, equalityFieldIds, partitionRecord, eqDeleteRowSchema, record); + } + + private void writeEqDeleteRecord( + Table table, + List equalityFieldIds, + Record partitionRecord, + Schema eqDeleteRowSchema, + Record deleteRecord) { + OutputFileFactory fileFactory = + OutputFileFactory.builderFor(table, 1, 1).format(FileFormat.PARQUET).build(); + GenericAppenderFactory appenderFactory = + new GenericAppenderFactory( + table.schema(), + table.spec(), + ArrayUtil.toIntArray(equalityFieldIds), + eqDeleteRowSchema, + null); + EncryptedOutputFile file = + createEncryptedOutputFile(createPartitionKey(table, partitionRecord), fileFactory); + EqualityDeleteWriter eqDeleteWriter = + appenderFactory.newEqDeleteWriter( + file, FileFormat.PARQUET, createPartitionKey(table, partitionRecord)); + try (EqualityDeleteWriter clsEqDeleteWriter = eqDeleteWriter) { + clsEqDeleteWriter.write(deleteRecord); + } catch (Exception e) { + throw new RuntimeException(e); + } + table.newRowDelta().addDeletes(eqDeleteWriter.toDeleteFile()).commit(); + } + + private PartitionKey createPartitionKey(Table table, Record record) { + if (table.spec().isUnpartitioned()) { + return null; + } + PartitionKey partitionKey = new PartitionKey(table.spec(), table.schema()); + partitionKey.partition(record); + return partitionKey; + } + + private EncryptedOutputFile createEncryptedOutputFile( + PartitionKey partition, OutputFileFactory fileFactory) { + if (partition == null) { + return fileFactory.newOutputFile(); + } else { + return fileFactory.newOutputFile(partition); + } + } + private SparkActions actions() { return SparkActions.get(); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java new file mode 100644 index 000000000000..8e6358f51bcd --- /dev/null +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.actions; + +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; + +public class TestSnapshotTableAction extends SparkCatalogTestBase { + private static final String SOURCE_NAME = "spark_catalog.default.source"; + + public TestSnapshotTableAction( + String catalogName, String implementation, Map config) { + super(catalogName, implementation, config); + } + + @After + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS %s PURGE", SOURCE_NAME); + } + + @Test + public void testSnapshotWithParallelTasks() throws IOException { + String location = temp.newFolder().toURI().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", SOURCE_NAME); + + AtomicInteger snapshotThreadsIndex = new AtomicInteger(0); + SparkActions.get() + .snapshotTable(SOURCE_NAME) + .as(tableName) + .executeWith( + Executors.newFixedThreadPool( + 4, + runnable -> { + Thread thread = new Thread(runnable); + thread.setName("table-snapshot-" + snapshotThreadsIndex.getAndIncrement()); + thread.setDaemon(true); + return thread; + })) + .execute(); + Assert.assertEquals(snapshotThreadsIndex.get(), 2); + } +} diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index b990a37cdd76..f56f1161e624 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -2020,8 +2020,13 @@ public void testFilesTablePartitionId() { .save(loadLocation(tableIdentifier)); List actual = - spark.read().format("iceberg").load(loadLocation(tableIdentifier, "files")) - .sort(DataFile.SPEC_ID.name()).collectAsList().stream() + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "files")) + .sort(DataFile.SPEC_ID.name()) + .collectAsList() + .stream() .map(r -> (Integer) r.getAs(DataFile.SPEC_ID.name())) .collect(Collectors.toList()); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java index 4643836542aa..29c2d4b39a1e 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java @@ -37,6 +37,7 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.Files; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Parameter; @@ -99,8 +100,6 @@ public class TestSparkReaderDeletes extends DeleteReadTests { protected static SparkSession spark = null; protected static HiveCatalog catalog = null; - @Parameter private String format; - @Parameter(index = 1) private boolean vectorized; @@ -110,10 +109,10 @@ public class TestSparkReaderDeletes extends DeleteReadTests { @Parameters(name = "format = {0}, vectorized = {1}, planningMode = {2}") public static Object[][] parameters() { return new Object[][] { - new Object[] {"parquet", false, PlanningMode.DISTRIBUTED}, - new Object[] {"parquet", true, PlanningMode.LOCAL}, - new Object[] {"orc", false, PlanningMode.DISTRIBUTED}, - new Object[] {"avro", false, PlanningMode.LOCAL} + new Object[] {FileFormat.PARQUET, false, PlanningMode.DISTRIBUTED}, + new Object[] {FileFormat.PARQUET, true, PlanningMode.LOCAL}, + new Object[] {FileFormat.ORC, false, PlanningMode.DISTRIBUTED}, + new Object[] {FileFormat.AVRO, false, PlanningMode.LOCAL} }; } @@ -169,17 +168,17 @@ protected Table createTable(String name, Schema schema, PartitionSpec spec) { ops.commit(meta, meta.upgradeToFormatVersion(2)); table .updateProperties() - .set(TableProperties.DEFAULT_FILE_FORMAT, format) + .set(TableProperties.DEFAULT_FILE_FORMAT, format.name()) .set(TableProperties.DATA_PLANNING_MODE, planningMode.modeName()) .set(TableProperties.DELETE_PLANNING_MODE, planningMode.modeName()) .commit(); - if (format.equals("parquet") || format.equals("orc")) { + if (format.equals(FileFormat.PARQUET) || format.equals(FileFormat.ORC)) { String vectorizationEnabled = - format.equals("parquet") + format.equals(FileFormat.PARQUET) ? TableProperties.PARQUET_VECTORIZATION_ENABLED : TableProperties.ORC_VECTORIZATION_ENABLED; String batchSize = - format.equals("parquet") + format.equals(FileFormat.PARQUET) ? TableProperties.PARQUET_BATCH_SIZE : TableProperties.ORC_BATCH_SIZE; table.updateProperties().set(vectorizationEnabled, String.valueOf(vectorized)).commit(); diff --git a/spark/v3.5/build.gradle b/spark/v3.5/build.gradle index 2ba5d493c6cd..e3c9ef4f0230 100644 --- a/spark/v3.5/build.gradle +++ b/spark/v3.5/build.gradle @@ -59,6 +59,7 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { implementation project(':iceberg-parquet') implementation project(':iceberg-arrow') implementation("org.scala-lang.modules:scala-collection-compat_${scalaVersion}:${libs.versions.scala.collection.compat.get()}") + implementation("org.apache.datasketches:datasketches-java:${libs.versions.datasketches.get()}") if (scalaVersion == '2.12') { // scala-collection-compat_2.12 pulls scala 2.12.17 and we need 2.12.18 for JDK 21 support implementation 'org.scala-lang:scala-library:2.12.18' @@ -192,7 +193,7 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer } project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersion}") { - apply plugin: 'io.github.goooler.shadow' + apply plugin: 'com.gradleup.shadow' tasks.jar.dependsOn tasks.shadowJar @@ -292,6 +293,7 @@ project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersio relocate 'com.carrotsearch', 'org.apache.iceberg.shaded.com.carrotsearch' relocate 'org.threeten.extra', 'org.apache.iceberg.shaded.org.threeten.extra' relocate 'org.roaringbitmap', 'org.apache.iceberg.shaded.org.roaringbitmap' + relocate 'org.apache.datasketches', 'org.apache.iceberg.shaded.org.apache.datasketches' archiveClassifier.set(null) } diff --git a/spark/v3.5/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java b/spark/v3.5/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java index e2ce5e956348..ed97e6b08414 100644 --- a/spark/v3.5/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java +++ b/spark/v3.5/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java @@ -20,49 +20,41 @@ import static org.apache.iceberg.PlanningMode.DISTRIBUTED; import static org.apache.iceberg.PlanningMode.LOCAL; -import static org.apache.spark.sql.functions.lit; import com.google.errorprone.annotations.FormatMethod; import com.google.errorprone.annotations.FormatString; import java.io.IOException; import java.io.UncheckedIOException; +import java.nio.ByteBuffer; import java.util.List; +import java.util.Map; import java.util.UUID; import java.util.concurrent.TimeUnit; import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.AppendFiles; import org.apache.iceberg.BatchScan; import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFiles; import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.FileMetadata; -import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.FileGenerationUtil; import org.apache.iceberg.PlanningMode; import org.apache.iceberg.RowDelta; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.ScanTask; -import org.apache.iceberg.Schema; import org.apache.iceberg.SparkDistributedDataScan; +import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestHelpers; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.spark.data.RandomData; import org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Types; import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.parser.ParseException; -import org.apache.spark.sql.types.StructType; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; @@ -108,10 +100,8 @@ public class PlanningBenchmark { Expressions.and(PARTITION_PREDICATE, SORT_KEY_PREDICATE); private static final int NUM_PARTITIONS = 30; - private static final int NUM_REAL_DATA_FILES_PER_PARTITION = 25; - private static final int NUM_REPLICA_DATA_FILES_PER_PARTITION = 50_000; + private static final int NUM_DATA_FILES_PER_PARTITION = 50_000; private static final int NUM_DELETE_FILES_PER_PARTITION = 50; - private static final int NUM_ROWS_PER_DATA_FILE = 500; private final Configuration hadoopConf = new Configuration(); private SparkSession spark; @@ -285,99 +275,43 @@ private void dropTable() { sql("DROP TABLE IF EXISTS %s PURGE", TABLE_NAME); } - private DataFile loadAddedDataFile() { - table.refresh(); - - Iterable dataFiles = table.currentSnapshot().addedDataFiles(table.io()); - return Iterables.getOnlyElement(dataFiles); - } - - private DeleteFile loadAddedDeleteFile() { - table.refresh(); - - Iterable deleteFiles = table.currentSnapshot().addedDeleteFiles(table.io()); - return Iterables.getOnlyElement(deleteFiles); - } - - private void initDataAndDeletes() throws NoSuchTableException { - Schema schema = table.schema(); - PartitionSpec spec = table.spec(); - LocationProvider locations = table.locationProvider(); - + private void initDataAndDeletes() { for (int partitionOrdinal = 0; partitionOrdinal < NUM_PARTITIONS; partitionOrdinal++) { - Dataset inputDF = - randomDataDF(schema, NUM_ROWS_PER_DATA_FILE) - .drop(PARTITION_COLUMN) - .withColumn(PARTITION_COLUMN, lit(partitionOrdinal)) - .drop(SORT_KEY_COLUMN) - .withColumn(SORT_KEY_COLUMN, lit(Integer.MIN_VALUE)); - - for (int fileOrdinal = 0; fileOrdinal < NUM_REAL_DATA_FILES_PER_PARTITION; fileOrdinal++) { - appendAsFile(inputDF); - } + StructLike partition = TestHelpers.Row.of(partitionOrdinal); - DataFile dataFile = loadAddedDataFile(); - - sql( - "DELETE FROM %s WHERE ss_item_sk IS NULL AND %s = %d", - TABLE_NAME, PARTITION_COLUMN, partitionOrdinal); - - DeleteFile deleteFile = loadAddedDeleteFile(); - - AppendFiles append = table.newFastAppend(); + RowDelta rowDelta = table.newRowDelta(); - for (int fileOrdinal = 0; fileOrdinal < NUM_REPLICA_DATA_FILES_PER_PARTITION; fileOrdinal++) { - String replicaFileName = UUID.randomUUID() + "-replica.parquet"; - DataFile replicaDataFile = - DataFiles.builder(spec) - .copy(dataFile) - .withPath(locations.newDataLocation(spec, dataFile.partition(), replicaFileName)) - .build(); - append.appendFile(replicaDataFile); + for (int fileOrdinal = 0; fileOrdinal < NUM_DATA_FILES_PER_PARTITION; fileOrdinal++) { + DataFile dataFile = generateDataFile(partition, Integer.MIN_VALUE, Integer.MIN_VALUE); + rowDelta.addRows(dataFile); } - append.commit(); - - RowDelta rowDelta = table.newRowDelta(); + // add one data file that would match the sort key predicate + DataFile sortKeyDataFile = generateDataFile(partition, SORT_KEY_VALUE, SORT_KEY_VALUE); + rowDelta.addRows(sortKeyDataFile); for (int fileOrdinal = 0; fileOrdinal < NUM_DELETE_FILES_PER_PARTITION; fileOrdinal++) { - String replicaFileName = UUID.randomUUID() + "-replica.parquet"; - DeleteFile replicaDeleteFile = - FileMetadata.deleteFileBuilder(spec) - .copy(deleteFile) - .withPath(locations.newDataLocation(spec, deleteFile.partition(), replicaFileName)) - .build(); - rowDelta.addDeletes(replicaDeleteFile); + DeleteFile deleteFile = FileGenerationUtil.generatePositionDeleteFile(table, partition); + rowDelta.addDeletes(deleteFile); } rowDelta.commit(); - - Dataset sortedInputDF = - randomDataDF(schema, NUM_ROWS_PER_DATA_FILE) - .drop(SORT_KEY_COLUMN) - .withColumn(SORT_KEY_COLUMN, lit(SORT_KEY_VALUE)) - .drop(PARTITION_COLUMN) - .withColumn(PARTITION_COLUMN, lit(partitionOrdinal)); - appendAsFile(sortedInputDF); } } - private void appendAsFile(Dataset df) throws NoSuchTableException { - df.coalesce(1).writeTo(TABLE_NAME).append(); + private DataFile generateDataFile(StructLike partition, int sortKeyMin, int sortKeyMax) { + int sortKeyFieldId = table.schema().findField(SORT_KEY_COLUMN).fieldId(); + ByteBuffer lower = Conversions.toByteBuffer(Types.IntegerType.get(), sortKeyMin); + Map lowerBounds = ImmutableMap.of(sortKeyFieldId, lower); + ByteBuffer upper = Conversions.toByteBuffer(Types.IntegerType.get(), sortKeyMax); + Map upperBounds = ImmutableMap.of(sortKeyFieldId, upper); + return FileGenerationUtil.generateDataFile(table, partition, lowerBounds, upperBounds); } private String newWarehouseDir() { return hadoopConf.get("hadoop.tmp.dir") + UUID.randomUUID(); } - private Dataset randomDataDF(Schema schema, int numRows) { - Iterable rows = RandomData.generateSpark(schema, numRows, 0); - JavaSparkContext context = JavaSparkContext.fromSparkContext(spark.sparkContext()); - JavaRDD rowRDD = context.parallelize(Lists.newArrayList(rows)); - StructType rowSparkType = SparkSchemaUtil.convert(schema); - return spark.internalCreateDataFrame(JavaRDD.toRDD(rowRDD), rowSparkType, false); - } - private List planFilesWithoutColumnStats(BatchScan scan, Expression predicate) { return planFiles(scan, predicate, false); } diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala index 2e438de2b8cd..6b1cc41da04c 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala @@ -226,11 +226,13 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS } val distributionMode = if (distributionSpec != null) { - DistributionMode.HASH - } else if (orderingSpec.UNORDERED != null || orderingSpec.LOCALLY != null) { - DistributionMode.NONE + Some(DistributionMode.HASH) + } else if (orderingSpec.UNORDERED != null) { + Some(DistributionMode.NONE) + } else if (orderingSpec.LOCALLY() != null) { + None } else { - DistributionMode.RANGE + Some(DistributionMode.RANGE) } val ordering = if (orderingSpec != null && orderingSpec.order != null) { diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala index feecc0235076..c9004ddc5bda 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.connector.catalog.TableCatalog case class SetWriteDistributionAndOrderingExec( catalog: TableCatalog, ident: Identifier, - distributionMode: DistributionMode, + distributionMode: Option[DistributionMode], sortOrder: Seq[(Term, SortDirection, NullOrder)]) extends LeafV2CommandExec { import CatalogV2Implicits._ @@ -56,9 +56,11 @@ case class SetWriteDistributionAndOrderingExec( } orderBuilder.commit() - txn.updateProperties() - .set(WRITE_DISTRIBUTION_MODE, distributionMode.modeName()) - .commit() + distributionMode.foreach { mode => + txn.updateProperties() + .set(WRITE_DISTRIBUTION_MODE, mode.modeName()) + .commit() + } txn.commitTransaction() diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java index 005d5e29d5ff..578845e3da2b 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java @@ -43,6 +43,8 @@ public static void startMetastoreAndSpark() { metastore.start(); TestBase.hiveConf = metastore.hiveConf(); + TestBase.spark.close(); + TestBase.spark = SparkSession.builder() .master("local[2]") diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java index bc60759bd038..3fd760c67c4a 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java @@ -436,4 +436,18 @@ public void testNetChangesWithComputeUpdates() { .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Not support net changes with update images"); } + + @TestTemplate + public void testUpdateWithInComparableType() { + sql( + "CREATE TABLE %s (id INT NOT NULL, data MAP, age INT) USING iceberg", + tableName); + + assertThatThrownBy( + () -> + sql("CALL %s.system.create_changelog_view(table => '%s')", catalogName, tableName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Identifier field is required as table contains unorderable columns: [data]"); + } } diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java index d258079541cd..42eb2af774e9 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java @@ -1401,6 +1401,28 @@ public void testDeleteToCustomWapBranchWithoutWhereClause() throws NoSuchTableEx }); } + @TestTemplate + public void testDeleteWithFilterOnNestedColumn() { + createAndInitNestedColumnsTable(); + + sql("INSERT INTO TABLE %s VALUES (1, named_struct(\"c1\", 3, \"c2\", \"v1\"))", tableName); + sql("INSERT INTO TABLE %s VALUES (2, named_struct(\"c1\", 2, \"c2\", \"v2\"))", tableName); + + sql("DELETE FROM %s WHERE complex.c1 > 3", tableName); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1), row(2)), + sql("SELECT id FROM %s order by id", tableName)); + + sql("DELETE FROM %s WHERE complex.c1 = 3", tableName); + assertEquals( + "Should have expected rows", ImmutableList.of(row(2)), sql("SELECT id FROM %s", tableName)); + + sql("DELETE FROM %s t WHERE t.complex.c1 = 2", tableName); + assertEquals( + "Should have expected rows", ImmutableList.of(), sql("SELECT id FROM %s", tableName)); + } + // TODO: multiple stripes for ORC protected void createAndInitPartitionedTable() { diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java index 08e6b1323c79..6b74391898e0 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java @@ -21,6 +21,8 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assumptions.assumeThat; +import java.sql.Timestamp; +import java.time.Instant; import java.time.LocalDateTime; import java.util.List; import org.apache.iceberg.Snapshot; @@ -246,6 +248,39 @@ public void testRollbackToTimestampWithoutExplicitCatalog() { sql("SELECT * FROM %s ORDER BY id", tableName)); } + @TestTemplate + public void testRollbackToTimestampBeforeOrEqualToOldestSnapshot() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot firstSnapshot = table.currentSnapshot(); + Timestamp beforeFirstSnapshot = + Timestamp.from(Instant.ofEpochMilli(firstSnapshot.timestampMillis() - 1)); + Timestamp exactFirstSnapshot = + Timestamp.from(Instant.ofEpochMilli(firstSnapshot.timestampMillis())); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rollback_to_timestamp(timestamp => TIMESTAMP '%s', table => '%s')", + catalogName, beforeFirstSnapshot, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot roll back, no valid snapshot older than: %s", + beforeFirstSnapshot.toInstant().toEpochMilli()); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rollback_to_timestamp(timestamp => TIMESTAMP '%s', table => '%s')", + catalogName, exactFirstSnapshot, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot roll back, no valid snapshot older than: %s", + exactFirstSnapshot.toInstant().toEpochMilli()); + } + @TestTemplate public void testInvalidRollbackToTimestampCases() { String timestamp = "TIMESTAMP '2007-12-03T10:15:30'"; diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetWriteDistributionAndOrdering.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetWriteDistributionAndOrdering.java index 77b7797fe192..b8547772da67 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetWriteDistributionAndOrdering.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetWriteDistributionAndOrdering.java @@ -200,8 +200,7 @@ public void testSetWriteLocallyOrdered() { table.refresh(); - String distributionMode = table.properties().get(TableProperties.WRITE_DISTRIBUTION_MODE); - assertThat(distributionMode).as("Distribution mode must match").isEqualTo("none"); + assertThat(table.properties().containsKey(TableProperties.WRITE_DISTRIBUTION_MODE)).isFalse(); SortOrder expected = SortOrder.builderFor(table.schema()) @@ -213,6 +212,25 @@ public void testSetWriteLocallyOrdered() { assertThat(table.sortOrder()).as("Sort order must match").isEqualTo(expected); } + @TestTemplate + public void testSetWriteLocallyOrderedToPartitionedTable() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, category string) USING iceberg PARTITIONED BY (id)", + tableName); + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.sortOrder().isUnsorted()).as("Table should start unsorted").isTrue(); + + sql("ALTER TABLE %s WRITE LOCALLY ORDERED BY category DESC", tableName); + + table.refresh(); + + assertThat(table.properties().containsKey(TableProperties.WRITE_DISTRIBUTION_MODE)).isFalse(); + + SortOrder expected = + SortOrder.builderFor(table.schema()).withOrderId(1).desc("category").build(); + assertThat(table.sortOrder()).as("Sort order must match").isEqualTo(expected); + } + @TestTemplate public void testSetWriteDistributedByWithSort() { sql( @@ -249,6 +267,13 @@ public void testSetWriteDistributedByWithLocalSort() { SortOrder expected = SortOrder.builderFor(table.schema()).withOrderId(1).asc("id").build(); assertThat(table.sortOrder()).as("Sort order must match").isEqualTo(expected); + + sql("ALTER TABLE %s WRITE LOCALLY ORDERED BY id", tableName); + + table.refresh(); + + String newDistributionMode = table.properties().get(TableProperties.WRITE_DISTRIBUTION_MODE); + assertThat(newDistributionMode).as("Distribution mode must match").isEqualTo(distributionMode); } @TestTemplate diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java index a4d0a2dfd3d0..6caff28bb16c 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java @@ -38,6 +38,7 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestSnapshotTableProcedure extends ExtensionsTestBase { private static final String SOURCE_NAME = "spark_catalog.default.source"; + // Currently we can only Snapshot only out of the Spark Session Catalog @AfterEach diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java index d521cc72cb31..61eafc5f4e78 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java @@ -23,6 +23,7 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; +import java.util.Locale; import java.util.Random; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -91,7 +92,7 @@ public static Object[][] parameters() { @TestTemplate public void readFromView() throws NoSuchTableException { insertRows(10); - String viewName = "simpleView"; + String viewName = viewName("simpleView"); String sql = String.format("SELECT id FROM %s", tableName); ViewCatalog viewCatalog = viewCatalog(); @@ -117,7 +118,7 @@ public void readFromView() throws NoSuchTableException { @TestTemplate public void readFromTrinoView() throws NoSuchTableException { insertRows(10); - String viewName = "trinoView"; + String viewName = viewName("trinoView"); String sql = String.format("SELECT id FROM %s", tableName); ViewCatalog viewCatalog = viewCatalog(); @@ -142,8 +143,8 @@ public void readFromTrinoView() throws NoSuchTableException { @TestTemplate public void readFromMultipleViews() throws NoSuchTableException { insertRows(6); - String viewName = "firstView"; - String secondView = "secondView"; + String viewName = viewName("firstView"); + String secondView = viewName("secondView"); String viewSQL = String.format("SELECT id FROM %s WHERE id <= 3", tableName); String secondViewSQL = String.format("SELECT id FROM %s WHERE id > 3", tableName); @@ -175,7 +176,7 @@ public void readFromMultipleViews() throws NoSuchTableException { @TestTemplate public void readFromViewUsingNonExistingTable() throws NoSuchTableException { insertRows(10); - String viewName = "viewWithNonExistingTable"; + String viewName = viewName("viewWithNonExistingTable"); ViewCatalog viewCatalog = viewCatalog(); Schema schema = new Schema(Types.NestedField.required(1, "id", Types.LongType.get())); @@ -199,7 +200,7 @@ public void readFromViewUsingNonExistingTable() throws NoSuchTableException { @TestTemplate public void readFromViewUsingNonExistingTableColumn() throws NoSuchTableException { insertRows(10); - String viewName = "viewWithNonExistingColumn"; + String viewName = viewName("viewWithNonExistingColumn"); ViewCatalog viewCatalog = viewCatalog(); Schema schema = new Schema(Types.NestedField.required(1, "non_existing", Types.LongType.get())); @@ -221,7 +222,7 @@ public void readFromViewUsingNonExistingTableColumn() throws NoSuchTableExceptio @TestTemplate public void readFromViewUsingInvalidSQL() throws NoSuchTableException { insertRows(10); - String viewName = "viewWithInvalidSQL"; + String viewName = viewName("viewWithInvalidSQL"); ViewCatalog viewCatalog = viewCatalog(); Schema schema = tableCatalog().loadTable(TableIdentifier.of(NAMESPACE, tableName)).schema(); @@ -243,7 +244,7 @@ public void readFromViewUsingInvalidSQL() throws NoSuchTableException { @TestTemplate public void readFromViewWithStaleSchema() throws NoSuchTableException { insertRows(10); - String viewName = "staleView"; + String viewName = viewName("staleView"); String sql = String.format("SELECT id, data FROM %s", tableName); ViewCatalog viewCatalog = viewCatalog(); @@ -269,7 +270,7 @@ public void readFromViewWithStaleSchema() throws NoSuchTableException { @TestTemplate public void readFromViewHiddenByTempView() throws NoSuchTableException { insertRows(10); - String viewName = "viewHiddenByTempView"; + String viewName = viewName("viewHiddenByTempView"); ViewCatalog viewCatalog = viewCatalog(); Schema schema = tableCatalog().loadTable(TableIdentifier.of(NAMESPACE, tableName)).schema(); @@ -296,7 +297,7 @@ public void readFromViewHiddenByTempView() throws NoSuchTableException { @TestTemplate public void readFromViewWithGlobalTempView() throws NoSuchTableException { insertRows(10); - String viewName = "viewWithGlobalTempView"; + String viewName = viewName("viewWithGlobalTempView"); String sql = String.format("SELECT id FROM %s WHERE id > 5", tableName); ViewCatalog viewCatalog = viewCatalog(); @@ -326,8 +327,8 @@ public void readFromViewWithGlobalTempView() throws NoSuchTableException { @TestTemplate public void readFromViewReferencingAnotherView() throws NoSuchTableException { insertRows(10); - String firstView = "viewBeingReferencedInAnotherView"; - String viewReferencingOtherView = "viewReferencingOtherView"; + String firstView = viewName("viewBeingReferencedInAnotherView"); + String viewReferencingOtherView = viewName("viewReferencingOtherView"); String firstSQL = String.format("SELECT id FROM %s WHERE id <= 5", tableName); String secondSQL = String.format("SELECT id FROM %s WHERE id > 4", firstView); @@ -357,8 +358,8 @@ public void readFromViewReferencingAnotherView() throws NoSuchTableException { @TestTemplate public void readFromViewReferencingTempView() throws NoSuchTableException { insertRows(10); - String tempView = "tempViewBeingReferencedInAnotherView"; - String viewReferencingTempView = "viewReferencingTempView"; + String tempView = viewName("tempViewBeingReferencedInAnotherView"); + String viewReferencingTempView = viewName("viewReferencingTempView"); String sql = String.format("SELECT id FROM %s", tempView); ViewCatalog viewCatalog = viewCatalog(); @@ -393,8 +394,8 @@ public void readFromViewReferencingTempView() throws NoSuchTableException { @TestTemplate public void readFromViewReferencingAnotherViewHiddenByTempView() throws NoSuchTableException { insertRows(10); - String innerViewName = "inner_view"; - String outerViewName = "outer_view"; + String innerViewName = viewName("inner_view"); + String outerViewName = viewName("outer_view"); String innerViewSQL = String.format("SELECT * FROM %s WHERE id > 5", tableName); String outerViewSQL = String.format("SELECT id FROM %s", innerViewName); @@ -441,8 +442,8 @@ public void readFromViewReferencingAnotherViewHiddenByTempView() throws NoSuchTa @TestTemplate public void readFromViewReferencingGlobalTempView() throws NoSuchTableException { insertRows(10); - String globalTempView = "globalTempViewBeingReferenced"; - String viewReferencingTempView = "viewReferencingGlobalTempView"; + String globalTempView = viewName("globalTempViewBeingReferenced"); + String viewReferencingTempView = viewName("viewReferencingGlobalTempView"); ViewCatalog viewCatalog = viewCatalog(); Schema schema = tableCatalog().loadTable(TableIdentifier.of(NAMESPACE, tableName)).schema(); @@ -480,7 +481,7 @@ public void readFromViewReferencingGlobalTempView() throws NoSuchTableException public void readFromViewReferencingTempFunction() throws NoSuchTableException { insertRows(10); String viewName = viewName("viewReferencingTempFunction"); - String functionName = "test_avg"; + String functionName = viewName("test_avg"); String sql = String.format("SELECT %s(id) FROM %s", functionName, tableName); sql( "CREATE TEMPORARY FUNCTION %s AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage'", @@ -511,7 +512,7 @@ public void readFromViewReferencingTempFunction() throws NoSuchTableException { @TestTemplate public void readFromViewWithCTE() throws NoSuchTableException { insertRows(10); - String viewName = "viewWithCTE"; + String viewName = viewName("viewWithCTE"); String sql = String.format( "WITH max_by_data AS (SELECT max(id) as max FROM %s) " @@ -533,7 +534,7 @@ public void readFromViewWithCTE() throws NoSuchTableException { @TestTemplate public void rewriteFunctionIdentifier() { - String viewName = "rewriteFunctionIdentifier"; + String viewName = viewName("rewriteFunctionIdentifier"); String sql = "SELECT iceberg_version() AS version"; assertThatThrownBy(() -> sql(sql)) @@ -559,7 +560,7 @@ public void rewriteFunctionIdentifier() { @TestTemplate public void builtinFunctionIdentifierNotRewritten() { - String viewName = "builtinFunctionIdentifierNotRewritten"; + String viewName = viewName("builtinFunctionIdentifierNotRewritten"); String sql = "SELECT trim(' abc ') AS result"; ViewCatalog viewCatalog = viewCatalog(); @@ -578,7 +579,7 @@ public void builtinFunctionIdentifierNotRewritten() { @TestTemplate public void rewriteFunctionIdentifierWithNamespace() { - String viewName = "rewriteFunctionIdentifierWithNamespace"; + String viewName = viewName("rewriteFunctionIdentifierWithNamespace"); String sql = "SELECT system.bucket(100, 'a') AS bucket_result, 'a' AS value"; ViewCatalog viewCatalog = viewCatalog(); @@ -605,7 +606,7 @@ public void rewriteFunctionIdentifierWithNamespace() { @TestTemplate public void fullFunctionIdentifier() { - String viewName = "fullFunctionIdentifier"; + String viewName = viewName("fullFunctionIdentifier"); String sql = String.format( "SELECT %s.system.bucket(100, 'a') AS bucket_result, 'a' AS value", catalogName); @@ -629,7 +630,7 @@ public void fullFunctionIdentifier() { @TestTemplate public void fullFunctionIdentifierNotRewrittenLoadFailure() { - String viewName = "fullFunctionIdentifierNotRewrittenLoadFailure"; + String viewName = viewName("fullFunctionIdentifierNotRewrittenLoadFailure"); String sql = "SELECT spark_catalog.system.bucket(100, 'a') AS bucket_result, 'a' AS value"; // avoid namespace failures @@ -821,7 +822,7 @@ public void renameViewTargetAlreadyExistsAsTable() { @TestTemplate public void dropView() { - String viewName = "viewToBeDropped"; + String viewName = viewName("viewToBeDropped"); String sql = String.format("SELECT id FROM %s", tableName); ViewCatalog viewCatalog = viewCatalog(); @@ -850,7 +851,7 @@ public void dropNonExistingView() { @TestTemplate public void dropViewIfExists() { - String viewName = "viewToBeDropped"; + String viewName = viewName("viewToBeDropped"); String sql = String.format("SELECT id FROM %s", tableName); ViewCatalog viewCatalog = viewCatalog(); @@ -875,7 +876,7 @@ public void dropViewIfExists() { /** The purpose of this test is mainly to make sure that normal view deletion isn't messed up */ @TestTemplate public void dropGlobalTempView() { - String globalTempView = "globalViewToBeDropped"; + String globalTempView = viewName("globalViewToBeDropped"); sql("CREATE GLOBAL TEMPORARY VIEW %s AS SELECT id FROM %s", globalTempView, tableName); assertThat(v1SessionCatalog().getGlobalTempView(globalTempView).isDefined()).isTrue(); @@ -886,7 +887,7 @@ public void dropGlobalTempView() { /** The purpose of this test is mainly to make sure that normal view deletion isn't messed up */ @TestTemplate public void dropTempView() { - String tempView = "tempViewToBeDropped"; + String tempView = viewName("tempViewToBeDropped"); sql("CREATE TEMPORARY VIEW %s AS SELECT id FROM %s", tempView, tableName); assertThat(v1SessionCatalog().getTempView(tempView).isDefined()).isTrue(); @@ -897,7 +898,7 @@ public void dropTempView() { /** The purpose of this test is mainly to make sure that normal view deletion isn't messed up */ @TestTemplate public void dropV1View() { - String v1View = "v1ViewToBeDropped"; + String v1View = viewName("v1ViewToBeDropped"); sql("USE spark_catalog"); sql("CREATE NAMESPACE IF NOT EXISTS %s", NAMESPACE); sql("CREATE TABLE %s (id INT, data STRING)", tableName); @@ -928,7 +929,7 @@ private String viewName(String viewName) { @TestTemplate public void createViewIfNotExists() { - String viewName = "viewThatAlreadyExists"; + String viewName = viewName("viewThatAlreadyExists"); sql("CREATE VIEW %s AS SELECT id FROM %s", viewName, tableName); assertThatThrownBy(() -> sql("CREATE VIEW %s AS SELECT id FROM %s", viewName, tableName)) @@ -969,8 +970,8 @@ public void createViewWithInvalidSQL() { @TestTemplate public void createViewReferencingTempView() throws NoSuchTableException { insertRows(10); - String tempView = "temporaryViewBeingReferencedInAnotherView"; - String viewReferencingTempView = "viewReferencingTemporaryView"; + String tempView = viewName("temporaryViewBeingReferencedInAnotherView"); + String viewReferencingTempView = viewName("viewReferencingTemporaryView"); sql("CREATE TEMPORARY VIEW %s AS SELECT id FROM %s WHERE id <= 5", tempView, tableName); @@ -988,8 +989,8 @@ public void createViewReferencingTempView() throws NoSuchTableException { @TestTemplate public void createViewReferencingGlobalTempView() throws NoSuchTableException { insertRows(10); - String globalTempView = "globalTemporaryViewBeingReferenced"; - String viewReferencingTempView = "viewReferencingGlobalTemporaryView"; + String globalTempView = viewName("globalTemporaryViewBeingReferenced"); + String viewReferencingTempView = viewName("viewReferencingGlobalTemporaryView"); sql( "CREATE GLOBAL TEMPORARY VIEW %s AS SELECT id FROM %s WHERE id <= 5", @@ -1012,7 +1013,7 @@ public void createViewReferencingGlobalTempView() throws NoSuchTableException { @TestTemplate public void createViewReferencingTempFunction() { String viewName = viewName("viewReferencingTemporaryFunction"); - String functionName = "test_avg_func"; + String functionName = viewName("test_avg_func"); sql( "CREATE TEMPORARY FUNCTION %s AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage'", @@ -1031,7 +1032,7 @@ public void createViewReferencingTempFunction() { @TestTemplate public void createViewReferencingQualifiedTempFunction() { String viewName = viewName("viewReferencingTemporaryFunction"); - String functionName = "test_avg_func_qualified"; + String functionName = viewName("test_avg_func_qualified"); sql( "CREATE TEMPORARY FUNCTION %s AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage'", @@ -1069,7 +1070,7 @@ public void createViewUsingNonExistingTable() { @TestTemplate public void createViewWithMismatchedColumnCounts() { - String viewName = "viewWithMismatchedColumnCounts"; + String viewName = viewName("viewWithMismatchedColumnCounts"); assertThatThrownBy( () -> sql("CREATE VIEW %s (id, data) AS SELECT id FROM %s", viewName, tableName)) @@ -1093,7 +1094,7 @@ public void createViewWithMismatchedColumnCounts() { @TestTemplate public void createViewWithColumnAliases() throws NoSuchTableException { insertRows(6); - String viewName = "viewWithColumnAliases"; + String viewName = viewName("viewWithColumnAliases"); sql( "CREATE VIEW %s (new_id COMMENT 'ID', new_data COMMENT 'DATA') AS SELECT id, data FROM %s WHERE id <= 3", @@ -1140,7 +1141,7 @@ public void createViewWithDuplicateColumnNames() { @TestTemplate public void createViewWithDuplicateQueryColumnNames() throws NoSuchTableException { insertRows(3); - String viewName = "viewWithDuplicateQueryColumnNames"; + String viewName = viewName("viewWithDuplicateQueryColumnNames"); String sql = String.format("SELECT id, id FROM %s WHERE id <= 3", tableName); // not specifying column aliases in the view should fail @@ -1158,7 +1159,7 @@ public void createViewWithDuplicateQueryColumnNames() throws NoSuchTableExceptio @TestTemplate public void createViewWithCTE() throws NoSuchTableException { insertRows(10); - String viewName = "simpleViewWithCTE"; + String viewName = viewName("simpleViewWithCTE"); String sql = String.format( "WITH max_by_data AS (SELECT max(id) as max FROM %s) " @@ -1173,8 +1174,8 @@ public void createViewWithCTE() throws NoSuchTableException { @TestTemplate public void createViewWithConflictingNamesForCTEAndTempView() throws NoSuchTableException { insertRows(10); - String viewName = "viewWithConflictingNamesForCTEAndTempView"; - String cteName = "cteName"; + String viewName = viewName("viewWithConflictingNamesForCTEAndTempView"); + String cteName = viewName("cteName"); String sql = String.format( "WITH %s AS (SELECT max(id) as max FROM %s) " @@ -1191,8 +1192,8 @@ public void createViewWithConflictingNamesForCTEAndTempView() throws NoSuchTable @TestTemplate public void createViewWithCTEReferencingTempView() { - String viewName = "viewWithCTEReferencingTempView"; - String tempViewInCTE = "tempViewInCTE"; + String viewName = viewName("viewWithCTEReferencingTempView"); + String tempViewInCTE = viewName("tempViewInCTE"); String sql = String.format( "WITH max_by_data AS (SELECT max(id) as max FROM %s) " @@ -1211,8 +1212,8 @@ public void createViewWithCTEReferencingTempView() { @TestTemplate public void createViewWithCTEReferencingTempFunction() { - String viewName = "viewWithCTEReferencingTempFunction"; - String functionName = "avg_function_in_cte"; + String viewName = viewName("viewWithCTEReferencingTempFunction"); + String functionName = viewName("avg_function_in_cte"); String sql = String.format( "WITH avg_data AS (SELECT %s(id) as avg FROM %s) " @@ -1245,8 +1246,8 @@ public void createViewWithNonExistingQueryColumn() { @TestTemplate public void createViewWithSubqueryExpressionUsingTempView() { - String viewName = "viewWithSubqueryExpression"; - String tempView = "simpleTempView"; + String viewName = viewName("viewWithSubqueryExpression"); + String tempView = viewName("simpleTempView"); String sql = String.format("SELECT * FROM %s WHERE id = (SELECT id FROM %s)", tableName, tempView); @@ -1262,8 +1263,8 @@ public void createViewWithSubqueryExpressionUsingTempView() { @TestTemplate public void createViewWithSubqueryExpressionUsingGlobalTempView() { - String viewName = "simpleViewWithSubqueryExpression"; - String globalTempView = "simpleGlobalTempView"; + String viewName = viewName("simpleViewWithSubqueryExpression"); + String globalTempView = viewName("simpleGlobalTempView"); String sql = String.format( "SELECT * FROM %s WHERE id = (SELECT id FROM global_temp.%s)", @@ -1284,7 +1285,7 @@ public void createViewWithSubqueryExpressionUsingGlobalTempView() { @TestTemplate public void createViewWithSubqueryExpressionUsingTempFunction() { String viewName = viewName("viewWithSubqueryExpression"); - String functionName = "avg_function_in_subquery"; + String functionName = viewName("avg_function_in_subquery"); String sql = String.format( "SELECT * FROM %s WHERE id < (SELECT %s(id) FROM %s)", @@ -1354,7 +1355,7 @@ public void createViewWithSubqueryExpressionInQueryThatIsRewritten() throws NoSu @TestTemplate public void describeView() { - String viewName = "describeView"; + String viewName = viewName("describeView"); sql("CREATE VIEW %s AS SELECT id, data FROM %s WHERE id <= 3", viewName, tableName); assertThat(sql("DESCRIBE %s", viewName)) @@ -1363,7 +1364,7 @@ public void describeView() { @TestTemplate public void describeExtendedView() { - String viewName = "describeExtendedView"; + String viewName = viewName("describeExtendedView"); String sql = String.format("SELECT id, data FROM %s WHERE id <= 3", tableName); sql( @@ -1388,7 +1389,7 @@ public void describeExtendedView() { @TestTemplate public void showViewProperties() { - String viewName = "showViewProps"; + String viewName = viewName("showViewProps"); sql( "CREATE VIEW %s TBLPROPERTIES ('key1'='val1', 'key2'='val2') AS SELECT id, data FROM %s WHERE id <= 3", @@ -1399,7 +1400,7 @@ public void showViewProperties() { @TestTemplate public void showViewPropertiesByKey() { - String viewName = "showViewPropsByKey"; + String viewName = viewName("showViewPropsByKey"); sql("CREATE VIEW %s AS SELECT id, data FROM %s WHERE id <= 3", viewName, tableName); assertThat(sql("SHOW TBLPROPERTIES %s", viewName)).contains(row("provider", "iceberg")); @@ -1420,39 +1421,43 @@ public void showViewPropertiesByKey() { public void showViews() throws NoSuchTableException { insertRows(6); String sql = String.format("SELECT * from %s", tableName); - sql("CREATE VIEW v1 AS %s", sql); - sql("CREATE VIEW prefixV2 AS %s", sql); - sql("CREATE VIEW prefixV3 AS %s", sql); - sql("CREATE GLOBAL TEMPORARY VIEW globalViewForListing AS %s", sql); - sql("CREATE TEMPORARY VIEW tempViewForListing AS %s", sql); + String v1 = viewName("v1"); + String prefixV2 = viewName("prefixV2"); + String prefixV3 = viewName("prefixV3"); + String globalViewForListing = viewName("globalViewForListing"); + String tempViewForListing = viewName("tempViewForListing"); + sql("CREATE VIEW %s AS %s", v1, sql); + sql("CREATE VIEW %s AS %s", prefixV2, sql); + sql("CREATE VIEW %s AS %s", prefixV3, sql); + sql("CREATE GLOBAL TEMPORARY VIEW %s AS %s", globalViewForListing, sql); + sql("CREATE TEMPORARY VIEW %s AS %s", tempViewForListing, sql); // spark stores temp views case-insensitive by default - Object[] tempView = row("", "tempviewforlisting", true); + Object[] tempView = row("", tempViewForListing.toLowerCase(Locale.ROOT), true); assertThat(sql("SHOW VIEWS")) .contains( - row(NAMESPACE.toString(), "prefixV2", false), - row(NAMESPACE.toString(), "prefixV3", false), - row(NAMESPACE.toString(), "v1", false), + row(NAMESPACE.toString(), prefixV2, false), + row(NAMESPACE.toString(), prefixV3, false), + row(NAMESPACE.toString(), v1, false), tempView); assertThat(sql("SHOW VIEWS IN %s", catalogName)) .contains( - row(NAMESPACE.toString(), "prefixV2", false), - row(NAMESPACE.toString(), "prefixV3", false), - row(NAMESPACE.toString(), "v1", false), + row(NAMESPACE.toString(), prefixV2, false), + row(NAMESPACE.toString(), prefixV3, false), + row(NAMESPACE.toString(), v1, false), tempView); assertThat(sql("SHOW VIEWS IN %s.%s", catalogName, NAMESPACE)) .contains( - row(NAMESPACE.toString(), "prefixV2", false), - row(NAMESPACE.toString(), "prefixV3", false), - row(NAMESPACE.toString(), "v1", false), + row(NAMESPACE.toString(), prefixV2, false), + row(NAMESPACE.toString(), prefixV3, false), + row(NAMESPACE.toString(), v1, false), tempView); assertThat(sql("SHOW VIEWS LIKE 'pref*'")) .contains( - row(NAMESPACE.toString(), "prefixV2", false), - row(NAMESPACE.toString(), "prefixV3", false)); + row(NAMESPACE.toString(), prefixV2, false), row(NAMESPACE.toString(), prefixV3, false)); assertThat(sql("SHOW VIEWS LIKE 'non-existing'")).isEmpty(); @@ -1461,7 +1466,7 @@ public void showViews() throws NoSuchTableException { assertThat(sql("SHOW VIEWS IN global_temp")) .contains( // spark stores temp views case-insensitive by default - row("global_temp", "globalviewforlisting", true), tempView); + row("global_temp", globalViewForListing.toLowerCase(Locale.ROOT), true), tempView); sql("USE spark_catalog"); assertThat(sql("SHOW VIEWS")).contains(tempView); @@ -1502,7 +1507,7 @@ public void showViewsWithCurrentNamespace() { @TestTemplate public void showCreateSimpleView() { - String viewName = "showCreateSimpleView"; + String viewName = viewName("showCreateSimpleView"); String sql = String.format("SELECT id, data FROM %s WHERE id <= 3", tableName); sql("CREATE VIEW %s AS %s", viewName, sql); @@ -1523,7 +1528,7 @@ public void showCreateSimpleView() { @TestTemplate public void showCreateComplexView() { - String viewName = "showCreateComplexView"; + String viewName = viewName("showCreateComplexView"); String sql = String.format("SELECT id, data FROM %s WHERE id <= 3", tableName); sql( @@ -1550,7 +1555,7 @@ public void showCreateComplexView() { @TestTemplate public void alterViewSetProperties() { - String viewName = "viewWithSetProperties"; + String viewName = viewName("viewWithSetProperties"); sql("CREATE VIEW %s AS SELECT id FROM %s WHERE id <= 3", viewName, tableName); @@ -1572,7 +1577,7 @@ public void alterViewSetProperties() { @TestTemplate public void alterViewSetReservedProperties() { - String viewName = "viewWithSetReservedProperties"; + String viewName = viewName("viewWithSetReservedProperties"); sql("CREATE VIEW %s AS SELECT id FROM %s WHERE id <= 3", viewName, tableName); @@ -1603,7 +1608,7 @@ public void alterViewSetReservedProperties() { @TestTemplate public void alterViewUnsetProperties() { - String viewName = "viewWithUnsetProperties"; + String viewName = viewName("viewWithUnsetProperties"); sql("CREATE VIEW %s AS SELECT id FROM %s WHERE id <= 3", viewName, tableName); ViewCatalog viewCatalog = viewCatalog(); @@ -1624,7 +1629,7 @@ public void alterViewUnsetProperties() { @TestTemplate public void alterViewUnsetUnknownProperty() { - String viewName = "viewWithUnsetUnknownProp"; + String viewName = viewName("viewWithUnsetUnknownProp"); sql("CREATE VIEW %s AS SELECT id FROM %s WHERE id <= 3", viewName, tableName); assertThatThrownBy(() -> sql("ALTER VIEW %s UNSET TBLPROPERTIES ('unknown-key')", viewName)) @@ -1638,7 +1643,7 @@ public void alterViewUnsetUnknownProperty() { @TestTemplate public void alterViewUnsetReservedProperties() { - String viewName = "viewWithUnsetReservedProperties"; + String viewName = viewName("viewWithUnsetReservedProperties"); sql("CREATE VIEW %s AS SELECT id FROM %s WHERE id <= 3", viewName, tableName); @@ -1721,7 +1726,7 @@ public void createOrReplaceViewWithColumnAliases() throws NoSuchTableException { @TestTemplate public void alterViewIsNotSupported() throws NoSuchTableException { insertRows(6); - String viewName = "alteredView"; + String viewName = viewName("alteredView"); sql("CREATE VIEW %s AS SELECT id, data FROM %s WHERE id <= 3", viewName, tableName); diff --git a/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java b/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java index 89f7ad5dc62f..ec445774a452 100644 --- a/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java +++ b/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java @@ -101,7 +101,7 @@ public void testAlterTable() { sql( "CREATE TABLE %s (category int, id bigint, data string, ts timestamp) USING iceberg", tableName); - Table table = getTable(); + Table table; // Add examples sql("ALTER TABLE %s ADD PARTITION FIELD bucket(16, id)", tableName); sql("ALTER TABLE %s ADD PARTITION FIELD truncate(data, 4)", tableName); diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java index c012d7a2af1f..fd0dcac6bbcc 100644 --- a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java +++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java @@ -22,6 +22,7 @@ import java.sql.Timestamp; import java.util.List; +import java.util.Locale; import java.util.UUID; import java.util.concurrent.TimeUnit; import org.apache.iceberg.AppendFiles; @@ -124,7 +125,7 @@ private void appendData() { for (int i = 0; i < NUM_SNAPSHOTS; i++) { AppendFiles appendFiles = table().newFastAppend(); for (int j = 0; j < NUM_FILES; j++) { - String path = String.format("%s/path/to/data-%d-%d.parquet", location, i, j); + String path = String.format(Locale.ROOT, "%s/path/to/data-%d-%d.parquet", location, i, j); validAndOrphanPaths.add(path); DataFile dataFile = DataFiles.builder(partitionSpec) diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java index f756c4cde015..99586f2503c2 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java @@ -52,6 +52,7 @@ public abstract class SparkContentFile implements ContentFile { private final int keyMetadataPosition; private final int splitOffsetsPosition; private final int sortOrderIdPosition; + private final int fileSpecIdPosition; private final int equalityIdsPosition; private final Type lowerBoundsType; private final Type upperBoundsType; @@ -100,6 +101,7 @@ public abstract class SparkContentFile implements ContentFile { this.keyMetadataPosition = positions.get(DataFile.KEY_METADATA.name()); this.splitOffsetsPosition = positions.get(DataFile.SPLIT_OFFSETS.name()); this.sortOrderIdPosition = positions.get(DataFile.SORT_ORDER_ID.name()); + this.fileSpecIdPosition = positions.get(DataFile.SPEC_ID.name()); this.equalityIdsPosition = positions.get(DataFile.EQUALITY_IDS.name()); } @@ -120,7 +122,10 @@ public Long pos() { @Override public int specId() { - return -1; + if (wrapped.isNullAt(fileSpecIdPosition)) { + return -1; + } + return wrapped.getAs(fileSpecIdPosition); } @Override diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java index 77ae796ffb76..9130e63ba97e 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java @@ -87,6 +87,10 @@ private SparkSQLProperties() {} "spark.sql.iceberg.executor-cache.max-total-size"; public static final long EXECUTOR_CACHE_MAX_TOTAL_SIZE_DEFAULT = 128 * 1024 * 1024; // 128 MB + // Controls whether to merge schema during write operation + public static final String MERGE_SCHEMA = "spark.sql.iceberg.merge-schema"; + public static final boolean MERGE_SCHEMA_DEFAULT = false; + public static final String EXECUTOR_CACHE_LOCALITY_ENABLED = "spark.sql.iceberg.executor-cache.locality.enabled"; public static final boolean EXECUTOR_CACHE_LOCALITY_ENABLED_DEFAULT = false; diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java index 7a96e97fb98a..c44969c49e39 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java @@ -443,7 +443,9 @@ public static void importSparkTable( * @param sourceTableIdent an identifier of the source Spark table * @param targetTable an Iceberg table where to import the data * @param stagingDir a staging directory to store temporary manifest files - * @param service executor service to use for file reading + * @param service executor service to use for file reading. If null, file reading will be + * performed on the current thread. * If non-null, the provided ExecutorService will be + * shutdown within this method after file reading is complete. */ public static void importSparkTable( SparkSession spark, @@ -501,7 +503,9 @@ public static void importSparkTable( * @param partitionFilter only import partitions whose values match those in the map, can be * partially defined * @param checkDuplicateFiles if true, throw exception if import results in a duplicate data file - * @param service executor service to use for file reading + * @param service executor service to use for file reading. If null, file reading will be + * performed on the current thread. If non-null, the provided ExecutorService will be shutdown + * within this method after file reading is complete. */ public static void importSparkTable( SparkSession spark, @@ -719,7 +723,9 @@ public static void importSparkPartitions( * @param spec a partition spec * @param stagingDir a staging directory to store temporary manifest files * @param checkDuplicateFiles if true, throw exception if import results in a duplicate data file - * @param service executor service to use for file reading + * @param service executor service to use for file reading. If null, file reading will be + * performed on the current thread. If non-null, the provided ExecutorService will be shutdown + * within this method after file reading is complete. */ public static void importSparkPartitions( SparkSession spark, @@ -851,6 +857,12 @@ private static void deleteManifests(FileIO io, List manifests) { .run(item -> io.deleteFile(item.path())); } + public static Dataset loadTable(SparkSession spark, Table table, long snapshotId) { + SparkTable sparkTable = new SparkTable(table, snapshotId, false); + DataSourceV2Relation relation = createRelation(sparkTable, ImmutableMap.of()); + return Dataset.ofRows(spark, relation); + } + public static Dataset loadMetadataTable( SparkSession spark, Table table, MetadataTableType type) { return loadMetadataTable(spark, table, type, ImmutableMap.of()); @@ -858,11 +870,16 @@ public static Dataset loadMetadataTable( public static Dataset loadMetadataTable( SparkSession spark, Table table, MetadataTableType type, Map extraOptions) { - SparkTable metadataTable = - new SparkTable(MetadataTableUtils.createMetadataTableInstance(table, type), false); + Table metadataTable = MetadataTableUtils.createMetadataTableInstance(table, type); + SparkTable sparkMetadataTable = new SparkTable(metadataTable, false); + DataSourceV2Relation relation = createRelation(sparkMetadataTable, extraOptions); + return Dataset.ofRows(spark, relation); + } + + private static DataSourceV2Relation createRelation( + SparkTable sparkTable, Map extraOptions) { CaseInsensitiveStringMap options = new CaseInsensitiveStringMap(extraOptions); - return Dataset.ofRows( - spark, DataSourceV2Relation.create(metadataTable, Some.empty(), Some.empty(), options)); + return DataSourceV2Relation.create(sparkTable, Option.empty(), Option.empty(), options); } /** diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java index 07393a67fe31..2c8c26d80977 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java @@ -141,7 +141,8 @@ public boolean mergeSchema() { .booleanConf() .option(SparkWriteOptions.MERGE_SCHEMA) .option(SparkWriteOptions.SPARK_MERGE_SCHEMA) - .defaultValue(SparkWriteOptions.MERGE_SCHEMA_DEFAULT) + .sessionConf(SparkSQLProperties.MERGE_SCHEMA) + .defaultValue(SparkSQLProperties.MERGE_SCHEMA_DEFAULT) .parse(); } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java index d9c4f66b192b..33db70bae587 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java @@ -68,7 +68,6 @@ private SparkWriteOptions() {} public static final String MERGE_SCHEMA = "merge-schema"; public static final String SPARK_MERGE_SCHEMA = "mergeSchema"; - public static final boolean MERGE_SCHEMA_DEFAULT = false; // Identifies snapshot from which to start validating conflicting changes public static final String VALIDATE_FROM_SNAPSHOT_ID = "validate-from-snapshot-id"; diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/ComputeTableStatsSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/ComputeTableStatsSparkAction.java new file mode 100644 index 000000000000..a508021c1040 --- /dev/null +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/ComputeTableStatsSparkAction.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.actions; + +import java.io.IOException; +import java.util.List; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.iceberg.GenericBlobMetadata; +import org.apache.iceberg.GenericStatisticsFile; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.IcebergBuild; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StatisticsFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.actions.ComputeTableStats; +import org.apache.iceberg.actions.ImmutableComputeTableStats; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.puffin.Blob; +import org.apache.iceberg.puffin.Puffin; +import org.apache.iceberg.puffin.PuffinWriter; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.spark.JobGroupInfo; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Computes the statistics of the given columns and stores it as Puffin files. */ +public class ComputeTableStatsSparkAction extends BaseSparkAction + implements ComputeTableStats { + + private static final Logger LOG = LoggerFactory.getLogger(ComputeTableStatsSparkAction.class); + private static final Result EMPTY_RESULT = ImmutableComputeTableStats.Result.builder().build(); + + private final Table table; + private List columns; + private Snapshot snapshot; + + ComputeTableStatsSparkAction(SparkSession spark, Table table) { + super(spark); + this.table = table; + this.snapshot = table.currentSnapshot(); + } + + @Override + protected ComputeTableStatsSparkAction self() { + return this; + } + + @Override + public ComputeTableStats columns(String... newColumns) { + Preconditions.checkArgument( + newColumns != null && newColumns.length > 0, "Columns cannot be null/empty"); + this.columns = ImmutableList.copyOf(ImmutableSet.copyOf(newColumns)); + return this; + } + + @Override + public ComputeTableStats snapshot(long newSnapshotId) { + Snapshot newSnapshot = table.snapshot(newSnapshotId); + Preconditions.checkArgument(newSnapshot != null, "Snapshot not found: %s", newSnapshotId); + this.snapshot = newSnapshot; + return this; + } + + @Override + public Result execute() { + if (snapshot == null) { + LOG.info("No snapshot to compute stats for table {}", table.name()); + return EMPTY_RESULT; + } + validateColumns(); + JobGroupInfo info = newJobGroupInfo("COMPUTE-TABLE-STATS", jobDesc()); + return withJobGroupInfo(info, this::doExecute); + } + + private Result doExecute() { + LOG.info( + "Computing stats for columns {} in {} (snapshot {})", + columns(), + table.name(), + snapshotId()); + List blobs = generateNDVBlobs(); + StatisticsFile statisticsFile = writeStatsFile(blobs); + table.updateStatistics().setStatistics(snapshotId(), statisticsFile).commit(); + return ImmutableComputeTableStats.Result.builder().statisticsFile(statisticsFile).build(); + } + + private StatisticsFile writeStatsFile(List blobs) { + LOG.info("Writing stats for table {} for snapshot {}", table.name(), snapshotId()); + OutputFile outputFile = table.io().newOutputFile(outputPath()); + try (PuffinWriter writer = Puffin.write(outputFile).createdBy(appIdentifier()).build()) { + blobs.forEach(writer::add); + writer.finish(); + return new GenericStatisticsFile( + snapshotId(), + outputFile.location(), + writer.fileSize(), + writer.footerSize(), + GenericBlobMetadata.from(writer.writtenBlobsMetadata())); + } catch (IOException e) { + throw new RuntimeIOException(e); + } + } + + private List generateNDVBlobs() { + return NDVSketchUtil.generateBlobs(spark(), table, snapshot, columns()); + } + + private List columns() { + if (columns == null) { + Schema schema = table.schemas().get(snapshot.schemaId()); + this.columns = + schema.columns().stream() + .filter(nestedField -> nestedField.type().isPrimitiveType()) + .map(Types.NestedField::name) + .collect(Collectors.toList()); + } + return columns; + } + + private void validateColumns() { + Schema schema = table.schemas().get(snapshot.schemaId()); + Preconditions.checkArgument(!columns().isEmpty(), "No columns found to compute stats"); + for (String columnName : columns()) { + Types.NestedField field = schema.findField(columnName); + Preconditions.checkArgument(field != null, "Can't find column %s in %s", columnName, schema); + Preconditions.checkArgument( + field.type().isPrimitiveType(), + "Can't compute stats on non-primitive type column: %s (%s)", + columnName, + field.type()); + } + } + + private String appIdentifier() { + String icebergVersion = IcebergBuild.fullVersion(); + String sparkVersion = spark().version(); + return String.format("Iceberg %s Spark %s", icebergVersion, sparkVersion); + } + + private long snapshotId() { + return snapshot.snapshotId(); + } + + private String jobDesc() { + return String.format( + "Computing table stats for %s (snapshot_id=%s, columns=%s)", + table.name(), snapshotId(), columns()); + } + + private String outputPath() { + TableOperations operations = ((HasTableOperations) table).operations(); + String fileName = String.format("%s-%s.stats", snapshotId(), UUID.randomUUID()); + return operations.metadataFileLocation(fileName); + } +} diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/NDVSketchUtil.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/NDVSketchUtil.java new file mode 100644 index 000000000000..c8a20d3cca73 --- /dev/null +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/NDVSketchUtil.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.actions; + +import java.nio.ByteBuffer; +import java.util.List; +import org.apache.datasketches.memory.Memory; +import org.apache.datasketches.theta.CompactSketch; +import org.apache.datasketches.theta.Sketch; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.puffin.Blob; +import org.apache.iceberg.puffin.PuffinCompressionCodec; +import org.apache.iceberg.puffin.StandardBlobTypes; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.SparkTableUtil; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.stats.ThetaSketchAgg; + +public class NDVSketchUtil { + + private NDVSketchUtil() {} + + public static final String APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY = "ndv"; + + static List generateBlobs( + SparkSession spark, Table table, Snapshot snapshot, List columns) { + Row sketches = computeNDVSketches(spark, table, snapshot, columns); + Schema schema = table.schemas().get(snapshot.schemaId()); + List blobs = Lists.newArrayList(); + for (int i = 0; i < columns.size(); i++) { + Types.NestedField field = schema.findField(columns.get(i)); + Sketch sketch = CompactSketch.wrap(Memory.wrap((byte[]) sketches.get(i))); + blobs.add(toBlob(field, sketch, snapshot)); + } + return blobs; + } + + private static Blob toBlob(Types.NestedField field, Sketch sketch, Snapshot snapshot) { + return new Blob( + StandardBlobTypes.APACHE_DATASKETCHES_THETA_V1, + ImmutableList.of(field.fieldId()), + snapshot.snapshotId(), + snapshot.sequenceNumber(), + ByteBuffer.wrap(sketch.toByteArray()), + PuffinCompressionCodec.ZSTD, + ImmutableMap.of( + APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY, + String.valueOf((long) sketch.getEstimate()))); + } + + private static Row computeNDVSketches( + SparkSession spark, Table table, Snapshot snapshot, List colNames) { + Dataset inputDF = SparkTableUtil.loadTable(spark, table, snapshot.snapshotId()); + return inputDF.select(toAggColumns(colNames)).first(); + } + + private static Column[] toAggColumns(List colNames) { + return colNames.stream().map(NDVSketchUtil::toAggColumn).toArray(Column[]::new); + } + + private static Column toAggColumn(String colName) { + ThetaSketchAgg agg = new ThetaSketchAgg(colName); + return new Column(agg.toAggregateExpression()); + } +} diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java new file mode 100644 index 000000000000..bbf65f58e19c --- /dev/null +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.actions; + +import static org.apache.spark.sql.functions.col; +import static org.apache.spark.sql.functions.min; + +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.Partitioning; +import org.apache.iceberg.RewriteFiles; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.ImmutableRemoveDanglingDeleteFiles; +import org.apache.iceberg.actions.RemoveDanglingDeleteFiles; +import org.apache.iceberg.spark.JobGroupInfo; +import org.apache.iceberg.spark.SparkDeleteFile; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.types.StructType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An action that removes dangling delete files from the current snapshot. A delete file is dangling + * if its deletes no longer applies to any live data files. + * + *

The following dangling delete files are removed: + * + *

    + *
  • Position delete files with a data sequence number less than that of any data file in the + * same partition + *
  • Equality delete files with a data sequence number less than or equal to that of any data + * file in the same partition + *
+ */ +class RemoveDanglingDeletesSparkAction + extends BaseSnapshotUpdateSparkAction + implements RemoveDanglingDeleteFiles { + + private static final Logger LOG = LoggerFactory.getLogger(RemoveDanglingDeletesSparkAction.class); + private final Table table; + + protected RemoveDanglingDeletesSparkAction(SparkSession spark, Table table) { + super(spark); + this.table = table; + } + + @Override + protected RemoveDanglingDeletesSparkAction self() { + return this; + } + + public Result execute() { + if (table.specs().size() == 1 && table.spec().isUnpartitioned()) { + // ManifestFilterManager already performs this table-wide delete on each commit + return ImmutableRemoveDanglingDeleteFiles.Result.builder() + .removedDeleteFiles(Collections.emptyList()) + .build(); + } + + String desc = String.format("Removing dangling delete files in %s", table.name()); + JobGroupInfo info = newJobGroupInfo("REMOVE-DELETES", desc); + return withJobGroupInfo(info, this::doExecute); + } + + Result doExecute() { + RewriteFiles rewriteFiles = table.newRewrite(); + List danglingDeletes = findDanglingDeletes(); + for (DeleteFile deleteFile : danglingDeletes) { + LOG.debug("Removing dangling delete file {}", deleteFile.path()); + rewriteFiles.deleteFile(deleteFile); + } + + if (!danglingDeletes.isEmpty()) { + commit(rewriteFiles); + } + + return ImmutableRemoveDanglingDeleteFiles.Result.builder() + .removedDeleteFiles(danglingDeletes) + .build(); + } + + /** + * Dangling delete files can be identified with following steps + * + *
    + *
  1. Group data files by partition keys and find the minimum data sequence number in each + * group. + *
  2. Left outer join delete files with partition-grouped data files on partition keys. + *
  3. Find dangling deletes by comparing each delete file's sequence number to its partition's + * minimum data sequence number. + *
  4. Collect results row to driver and use {@link SparkDeleteFile SparkDeleteFile} to wrap + * rows to valid delete files + *
+ */ + private List findDanglingDeletes() { + Dataset minSequenceNumberByPartition = + loadMetadataTable(table, MetadataTableType.ENTRIES) + // find live data files + .filter("data_file.content == 0 AND status < 2") + .selectExpr( + "data_file.partition as partition", + "data_file.spec_id as spec_id", + "sequence_number") + .groupBy("partition", "spec_id") + .agg(min("sequence_number")) + .toDF("grouped_partition", "grouped_spec_id", "min_data_sequence_number"); + + Dataset deleteEntries = + loadMetadataTable(table, MetadataTableType.ENTRIES) + // find live delete files + .filter("data_file.content != 0 AND status < 2"); + + Column joinOnPartition = + deleteEntries + .col("data_file.spec_id") + .equalTo(minSequenceNumberByPartition.col("grouped_spec_id")) + .and( + deleteEntries + .col("data_file.partition") + .equalTo(minSequenceNumberByPartition.col("grouped_partition"))); + + Column filterOnDanglingDeletes = + col("min_data_sequence_number") + // delete fies without any data files in partition + .isNull() + // position delete files without any applicable data files in partition + .or( + col("data_file.content") + .equalTo("1") + .and(col("sequence_number").$less(col("min_data_sequence_number")))) + // equality delete files without any applicable data files in the partition + .or( + col("data_file.content") + .equalTo("2") + .and(col("sequence_number").$less$eq(col("min_data_sequence_number")))); + + Dataset danglingDeletes = + deleteEntries + .join(minSequenceNumberByPartition, joinOnPartition, "left") + .filter(filterOnDanglingDeletes) + .select("data_file.*"); + return danglingDeletes.collectAsList().stream() + // map on driver because SparkDeleteFile is not serializable + .map(row -> deleteFileWrapper(danglingDeletes.schema(), row)) + .collect(Collectors.toList()); + } + + private DeleteFile deleteFileWrapper(StructType sparkFileType, Row row) { + int specId = row.getInt(row.fieldIndex("spec_id")); + Types.StructType combinedFileType = DataFile.getType(Partitioning.partitionType(table)); + // Set correct spec id + Types.StructType projection = DataFile.getType(table.specs().get(specId).partitionType()); + return new SparkDeleteFile(combinedFileType, projection, sparkFileType).wrap(row); + } +} diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java index d33e5e540893..4e381a7bd362 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java @@ -40,6 +40,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.actions.FileRewriter; import org.apache.iceberg.actions.ImmutableRewriteDataFiles; +import org.apache.iceberg.actions.ImmutableRewriteDataFiles.Result.Builder; import org.apache.iceberg.actions.RewriteDataFiles; import org.apache.iceberg.actions.RewriteDataFilesCommitManager; import org.apache.iceberg.actions.RewriteFileGroup; @@ -53,6 +54,7 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Queues; @@ -83,7 +85,8 @@ public class RewriteDataFilesSparkAction TARGET_FILE_SIZE_BYTES, USE_STARTING_SEQUENCE_NUMBER, REWRITE_JOB_ORDER, - OUTPUT_SPEC_ID); + OUTPUT_SPEC_ID, + REMOVE_DANGLING_DELETES); private static final RewriteDataFilesSparkAction.Result EMPTY_RESULT = ImmutableRewriteDataFiles.Result.builder().rewriteResults(ImmutableList.of()).build(); @@ -95,6 +98,7 @@ public class RewriteDataFilesSparkAction private int maxCommits; private int maxFailedCommits; private boolean partialProgressEnabled; + private boolean removeDanglingDeletes; private boolean useStartingSequenceNumber; private RewriteJobOrder rewriteJobOrder; private FileRewriter rewriter = null; @@ -175,11 +179,18 @@ public RewriteDataFiles.Result execute() { Stream groupStream = toGroupStream(ctx, fileGroupsByPartition); - if (partialProgressEnabled) { - return doExecuteWithPartialProgress(ctx, groupStream, commitManager(startingSnapshotId)); - } else { - return doExecute(ctx, groupStream, commitManager(startingSnapshotId)); + Builder resultBuilder = + partialProgressEnabled + ? doExecuteWithPartialProgress(ctx, groupStream, commitManager(startingSnapshotId)) + : doExecute(ctx, groupStream, commitManager(startingSnapshotId)); + + if (removeDanglingDeletes) { + RemoveDanglingDeletesSparkAction action = + new RemoveDanglingDeletesSparkAction(spark(), table); + int removedCount = Iterables.size(action.execute().removedDeleteFiles()); + resultBuilder.removedDeleteFilesCount(removedCount); } + return resultBuilder.build(); } StructLikeMap>> planFileGroups(long startingSnapshotId) { @@ -264,7 +275,7 @@ RewriteDataFilesCommitManager commitManager(long startingSnapshotId) { table, startingSnapshotId, useStartingSequenceNumber, commitSummary()); } - private Result doExecute( + private Builder doExecute( RewriteExecutionContext ctx, Stream groupStream, RewriteDataFilesCommitManager commitManager) { @@ -326,10 +337,10 @@ private Result doExecute( List rewriteResults = rewrittenGroups.stream().map(RewriteFileGroup::asResult).collect(Collectors.toList()); - return ImmutableRewriteDataFiles.Result.builder().rewriteResults(rewriteResults).build(); + return ImmutableRewriteDataFiles.Result.builder().rewriteResults(rewriteResults); } - private Result doExecuteWithPartialProgress( + private Builder doExecuteWithPartialProgress( RewriteExecutionContext ctx, Stream groupStream, RewriteDataFilesCommitManager commitManager) { @@ -386,8 +397,7 @@ private Result doExecuteWithPartialProgress( return ImmutableRewriteDataFiles.Result.builder() .rewriteResults(toRewriteResults(commitService.results())) - .rewriteFailures(rewriteFailures) - .build(); + .rewriteFailures(rewriteFailures); } Stream toGroupStream( @@ -456,6 +466,10 @@ void validateAndInitOptions() { PropertyUtil.propertyAsBoolean( options(), USE_STARTING_SEQUENCE_NUMBER, USE_STARTING_SEQUENCE_NUMBER_DEFAULT); + removeDanglingDeletes = + PropertyUtil.propertyAsBoolean( + options(), REMOVE_DANGLING_DELETES, REMOVE_DANGLING_DELETES_DEFAULT); + rewriteJobOrder = RewriteJobOrder.fromName( PropertyUtil.propertyAsString(options(), REWRITE_JOB_ORDER, REWRITE_JOB_ORDER_DEFAULT)); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java index fb67ded96e35..ba9fa2e7b4db 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java @@ -20,6 +20,8 @@ import org.apache.iceberg.Table; import org.apache.iceberg.actions.ActionsProvider; +import org.apache.iceberg.actions.ComputeTableStats; +import org.apache.iceberg.actions.RemoveDanglingDeleteFiles; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.Spark3Util.CatalogAndIdentifier; import org.apache.spark.sql.SparkSession; @@ -96,4 +98,14 @@ public DeleteReachableFilesSparkAction deleteReachableFiles(String metadataLocat public RewritePositionDeleteFilesSparkAction rewritePositionDeletes(Table table) { return new RewritePositionDeleteFilesSparkAction(spark, table); } + + @Override + public ComputeTableStats computeTableStats(Table table) { + return new ComputeTableStatsSparkAction(spark, table); + } + + @Override + public RemoveDanglingDeleteFiles removeDanglingDeleteFiles(Table table) { + return new RemoveDanglingDeletesSparkAction(spark, table); + } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroReader.java index 4622d2928ac4..7d92d963a9f4 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroReader.java @@ -37,16 +37,28 @@ import org.apache.iceberg.types.Types; import org.apache.spark.sql.catalyst.InternalRow; +/** + * @deprecated will be removed in 1.8.0; use SparkPlannedAvroReader instead. + */ +@Deprecated public class SparkAvroReader implements DatumReader, SupportsRowPosition { private final Schema readSchema; private final ValueReader reader; private Schema fileSchema = null; + /** + * @deprecated will be removed in 1.8.0; use SparkPlannedAvroReader instead. + */ + @Deprecated public SparkAvroReader(org.apache.iceberg.Schema expectedSchema, Schema readSchema) { this(expectedSchema, readSchema, ImmutableMap.of()); } + /** + * @deprecated will be removed in 1.8.0; use SparkPlannedAvroReader instead. + */ + @Deprecated @SuppressWarnings("unchecked") public SparkAvroReader( org.apache.iceberg.Schema expectedSchema, Schema readSchema, Map constants) { diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkPlannedAvroReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkPlannedAvroReader.java new file mode 100644 index 000000000000..dc4af24685b3 --- /dev/null +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkPlannedAvroReader.java @@ -0,0 +1,190 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.data; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.function.Supplier; +import org.apache.avro.LogicalType; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.io.DatumReader; +import org.apache.avro.io.Decoder; +import org.apache.iceberg.avro.AvroWithPartnerVisitor; +import org.apache.iceberg.avro.SupportsRowPosition; +import org.apache.iceberg.avro.ValueReader; +import org.apache.iceberg.avro.ValueReaders; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; +import org.apache.spark.sql.catalyst.InternalRow; + +public class SparkPlannedAvroReader implements DatumReader, SupportsRowPosition { + + private final Types.StructType expectedType; + private final Map idToConstant; + private ValueReader reader; + + public static SparkPlannedAvroReader create(org.apache.iceberg.Schema schema) { + return create(schema, ImmutableMap.of()); + } + + public static SparkPlannedAvroReader create( + org.apache.iceberg.Schema schema, Map constants) { + return new SparkPlannedAvroReader(schema, constants); + } + + private SparkPlannedAvroReader( + org.apache.iceberg.Schema expectedSchema, Map constants) { + this.expectedType = expectedSchema.asStruct(); + this.idToConstant = constants; + } + + @Override + @SuppressWarnings("unchecked") + public void setSchema(Schema fileSchema) { + this.reader = + (ValueReader) + AvroWithPartnerVisitor.visit( + expectedType, + fileSchema, + new ReadBuilder(idToConstant), + AvroWithPartnerVisitor.FieldIDAccessors.get()); + } + + @Override + public InternalRow read(InternalRow reuse, Decoder decoder) throws IOException { + return reader.read(decoder, reuse); + } + + @Override + public void setRowPositionSupplier(Supplier posSupplier) { + if (reader instanceof SupportsRowPosition) { + ((SupportsRowPosition) reader).setRowPositionSupplier(posSupplier); + } + } + + private static class ReadBuilder extends AvroWithPartnerVisitor> { + private final Map idToConstant; + + private ReadBuilder(Map idToConstant) { + this.idToConstant = idToConstant; + } + + @Override + public ValueReader record(Type partner, Schema record, List> fieldReaders) { + if (partner == null) { + return ValueReaders.skipStruct(fieldReaders); + } + + Types.StructType expected = partner.asStructType(); + List>> readPlan = + ValueReaders.buildReadPlan(expected, record, fieldReaders, idToConstant); + + // TODO: should this pass expected so that struct.get can reuse containers? + return SparkValueReaders.struct(readPlan, expected.fields().size()); + } + + @Override + public ValueReader union(Type partner, Schema union, List> options) { + return ValueReaders.union(options); + } + + @Override + public ValueReader array(Type partner, Schema array, ValueReader elementReader) { + return SparkValueReaders.array(elementReader); + } + + @Override + public ValueReader arrayMap( + Type partner, Schema map, ValueReader keyReader, ValueReader valueReader) { + return SparkValueReaders.arrayMap(keyReader, valueReader); + } + + @Override + public ValueReader map(Type partner, Schema map, ValueReader valueReader) { + return SparkValueReaders.map(SparkValueReaders.strings(), valueReader); + } + + @Override + public ValueReader primitive(Type partner, Schema primitive) { + LogicalType logicalType = primitive.getLogicalType(); + if (logicalType != null) { + switch (logicalType.getName()) { + case "date": + // Spark uses the same representation + return ValueReaders.ints(); + + case "timestamp-millis": + // adjust to microseconds + ValueReader longs = ValueReaders.longs(); + return (ValueReader) (decoder, ignored) -> longs.read(decoder, null) * 1000L; + + case "timestamp-micros": + // Spark uses the same representation + return ValueReaders.longs(); + + case "decimal": + return SparkValueReaders.decimal( + ValueReaders.decimalBytesReader(primitive), + ((LogicalTypes.Decimal) logicalType).getScale()); + + case "uuid": + return SparkValueReaders.uuids(); + + default: + throw new IllegalArgumentException("Unknown logical type: " + logicalType); + } + } + + switch (primitive.getType()) { + case NULL: + return ValueReaders.nulls(); + case BOOLEAN: + return ValueReaders.booleans(); + case INT: + if (partner != null && partner.typeId() == Type.TypeID.LONG) { + return ValueReaders.intsAsLongs(); + } + return ValueReaders.ints(); + case LONG: + return ValueReaders.longs(); + case FLOAT: + if (partner != null && partner.typeId() == Type.TypeID.DOUBLE) { + return ValueReaders.floatsAsDoubles(); + } + return ValueReaders.floats(); + case DOUBLE: + return ValueReaders.doubles(); + case STRING: + return SparkValueReaders.strings(); + case FIXED: + return ValueReaders.fixed(primitive.getFixedSize()); + case BYTES: + return ValueReaders.bytes(); + case ENUM: + return SparkValueReaders.enums(primitive.getEnumSymbols()); + default: + throw new IllegalArgumentException("Unsupported type: " + primitive); + } + } + } +} diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueReaders.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueReaders.java index 3cbf38d88bf4..7e65535f5ecb 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueReaders.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueReaders.java @@ -32,6 +32,7 @@ import org.apache.iceberg.avro.ValueReaders; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.UUIDUtil; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; @@ -74,6 +75,11 @@ static ValueReader map(ValueReader keyReader, ValueReader< return new MapReader(keyReader, valueReader); } + static ValueReader struct( + List>> readPlan, int numFields) { + return new PlannedStructReader(readPlan, numFields); + } + static ValueReader struct( List> readers, Types.StructType struct, Map idToConstant) { return new StructReader(readers, struct, idToConstant); @@ -249,6 +255,38 @@ public ArrayBasedMapData read(Decoder decoder, Object reuse) throws IOException } } + static class PlannedStructReader extends ValueReaders.PlannedStructReader { + private final int numFields; + + protected PlannedStructReader(List>> readPlan, int numFields) { + super(readPlan); + this.numFields = numFields; + } + + @Override + protected InternalRow reuseOrCreate(Object reuse) { + if (reuse instanceof GenericInternalRow + && ((GenericInternalRow) reuse).numFields() == numFields) { + return (InternalRow) reuse; + } + return new GenericInternalRow(numFields); + } + + @Override + protected Object get(InternalRow struct, int pos) { + return null; + } + + @Override + protected void set(InternalRow struct, int pos, Object value) { + if (value != null) { + struct.update(pos, value); + } else { + struct.setNullAt(pos); + } + } + } + static class StructReader extends ValueReaders.StructReader { private final int numFields; diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java index b4594d91c0ef..ae77b69133f3 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java @@ -23,6 +23,7 @@ import java.util.Map; import java.util.Set; import java.util.function.Predicate; +import java.util.stream.Collectors; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -37,6 +38,7 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.OrderUtils; import org.apache.spark.sql.connector.catalog.Identifier; import org.apache.spark.sql.connector.catalog.TableCatalog; import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter; @@ -146,10 +148,21 @@ public InternalRow[] call(InternalRow args) { Dataset df = loadRows(changelogTableIdent, options(input)); boolean netChanges = input.asBoolean(NET_CHANGES, false); + String[] identifierColumns = identifierColumns(input, tableIdent); + Set unorderableColumnNames = + Arrays.stream(df.schema().fields()) + .filter(field -> !OrderUtils.isOrderable(field.dataType())) + .map(StructField::name) + .collect(Collectors.toSet()); + + Preconditions.checkArgument( + identifierColumns.length > 0 || unorderableColumnNames.isEmpty(), + "Identifier field is required as table contains unorderable columns: %s", + unorderableColumnNames); if (shouldComputeUpdateImages(input)) { Preconditions.checkArgument(!netChanges, "Not support net changes with update images"); - df = computeUpdateImages(identifierColumns(input, tableIdent), df); + df = computeUpdateImages(identifierColumns, df); } else { df = removeCarryoverRows(df, netChanges); } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseRowReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseRowReader.java index 927084caea1c..eb97185e21f1 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseRowReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseRowReader.java @@ -32,9 +32,9 @@ import org.apache.iceberg.orc.ORC; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.spark.data.SparkAvroReader; import org.apache.iceberg.spark.data.SparkOrcReader; import org.apache.iceberg.spark.data.SparkParquetReaders; +import org.apache.iceberg.spark.data.SparkPlannedAvroReader; import org.apache.iceberg.types.TypeUtil; import org.apache.spark.sql.catalyst.InternalRow; @@ -77,7 +77,7 @@ private CloseableIterable newAvroIterable( .reuseContainers() .project(projection) .split(start, length) - .createReaderFunc(readSchema -> new SparkAvroReader(projection, readSchema, idToConstant)) + .createReaderFunc(readSchema -> SparkPlannedAvroReader.create(projection, idToConstant)) .withNameMapping(nameMapping()) .build(); } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java index 9df12fc060ae..c822ed743f85 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java @@ -49,6 +49,10 @@ import org.apache.spark.sql.types.StructType; import org.apache.spark.unsafe.types.UTF8String; +/** + * @deprecated since 1.7.0, will be removed in 1.8.0; use {@link SparkFileWriterFactory} instead. + */ +@Deprecated class SparkAppenderFactory implements FileAppenderFactory { private final Map properties; private final Schema writeSchema; diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java index d91779475845..73e6ab01563c 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java @@ -36,11 +36,11 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.OutputFileFactory; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator; import org.apache.iceberg.spark.ScanTaskSetManager; import org.apache.iceberg.spark.SparkWriteConf; +import org.apache.iceberg.util.DeleteFileSet; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.broadcast.Broadcast; import org.apache.spark.sql.SparkSession; @@ -148,7 +148,7 @@ public boolean useCommitCoordinator() { @Override public void commit(WriterCommitMessage[] messages) { PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get(); - coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages))); + coordinator.stageRewrite(table, fileSetId, DeleteFileSet.of(files(messages))); } @Override diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java index 8b88cf49c692..019f3919dc57 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java @@ -198,25 +198,31 @@ protected Statistics estimateStatistics(Snapshot snapshot) { if (!files.isEmpty()) { List metadataList = (files.get(0)).blobMetadata(); - for (BlobMetadata blobMetadata : metadataList) { - int id = blobMetadata.fields().get(0); - String colName = table.schema().findColumnName(id); + Map> groupedByField = + metadataList.stream() + .collect( + Collectors.groupingBy( + metadata -> metadata.fields().get(0), Collectors.toList())); + + for (Map.Entry> entry : groupedByField.entrySet()) { + String colName = table.schema().findColumnName(entry.getKey()); NamedReference ref = FieldReference.column(colName); - Long ndv = null; - if (blobMetadata - .type() - .equals(org.apache.iceberg.puffin.StandardBlobTypes.APACHE_DATASKETCHES_THETA_V1)) { - String ndvStr = blobMetadata.properties().get(NDV_KEY); - if (!Strings.isNullOrEmpty(ndvStr)) { - ndv = Long.parseLong(ndvStr); + + for (BlobMetadata blobMetadata : entry.getValue()) { + if (blobMetadata + .type() + .equals(org.apache.iceberg.puffin.StandardBlobTypes.APACHE_DATASKETCHES_THETA_V1)) { + String ndvStr = blobMetadata.properties().get(NDV_KEY); + if (!Strings.isNullOrEmpty(ndvStr)) { + ndv = Long.parseLong(ndvStr); + } else { + LOG.debug("{} is not set in BlobMetadata for column {}", NDV_KEY, colName); + } } else { - LOG.debug("ndv is not set in BlobMetadata for column {}", colName); + LOG.debug("Blob type {} is not supported yet", blobMetadata.type()); } - } else { - LOG.debug("DataSketch blob is not available for column {}", colName); } - ColumnStatistics colStats = new SparkColumnStatistics(ndv, null, null, null, null, null, null); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java index 9dc214a755d3..d511fefd8ae0 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java @@ -49,7 +49,6 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.metrics.InMemoryMetricsReporter; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.Spark3Util; @@ -239,8 +238,7 @@ public boolean pushAggregation(Aggregation aggregation) { buildIcebergBatchScan(true /* include Column Stats */, schemaWithMetadataColumns()); try (CloseableIterable fileScanTasks = scan.planFiles()) { - List tasks = ImmutableList.copyOf(fileScanTasks); - for (FileScanTask task : tasks) { + for (FileScanTask task : fileScanTasks) { if (!task.deletes().isEmpty()) { LOG.info("Skipping aggregate pushdown: detected row level deletes"); return false; diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java index e4a0eb700be6..cc3dc592ecee 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java @@ -52,12 +52,12 @@ import org.apache.iceberg.io.PartitioningWriter; import org.apache.iceberg.io.RollingDataWriter; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.CommitMetadata; import org.apache.iceberg.spark.FileRewriteCoordinator; import org.apache.iceberg.spark.SparkWriteConf; import org.apache.iceberg.spark.SparkWriteRequirements; +import org.apache.iceberg.util.DataFileSet; import org.apache.spark.TaskContext; import org.apache.spark.TaskContext$; import org.apache.spark.api.java.JavaSparkContext; @@ -491,7 +491,7 @@ private RewriteFiles(String fileSetID) { @Override public void commit(WriterCommitMessage[] messages) { FileRewriteCoordinator coordinator = FileRewriteCoordinator.get(); - coordinator.stageRewrite(table, fileSetID, ImmutableSet.copyOf(files(messages))); + coordinator.stageRewrite(table, fileSetID, DataFileSet.of(files(messages))); } } diff --git a/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetWriteDistributionAndOrdering.scala b/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetWriteDistributionAndOrdering.scala index 0a0234cdfe34..7b599eb3da1d 100644 --- a/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetWriteDistributionAndOrdering.scala +++ b/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetWriteDistributionAndOrdering.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits case class SetWriteDistributionAndOrdering( table: Seq[String], - distributionMode: DistributionMode, + distributionMode: Option[DistributionMode], sortOrder: Seq[(Term, SortDirection, NullOrder)]) extends LeafCommand { import CatalogV2Implicits._ diff --git a/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala b/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala new file mode 100644 index 000000000000..cca16960f434 --- /dev/null +++ b/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.stats + +import java.nio.ByteBuffer +import org.apache.datasketches.common.Family +import org.apache.datasketches.memory.Memory +import org.apache.datasketches.theta.CompactSketch +import org.apache.datasketches.theta.SetOperationBuilder +import org.apache.datasketches.theta.Sketch +import org.apache.datasketches.theta.UpdateSketch +import org.apache.iceberg.spark.SparkSchemaUtil +import org.apache.iceberg.types.Conversions +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.expressions.aggregate.ImperativeAggregate +import org.apache.spark.sql.catalyst.expressions.aggregate.TypedImperativeAggregate +import org.apache.spark.sql.catalyst.trees.UnaryLike +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.types.BinaryType +import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.types.Decimal +import org.apache.spark.unsafe.types.UTF8String + +/** + * ThetaSketchAgg generates Alpha family sketch with default seed. + * The values fed to the sketch are converted to bytes using Iceberg's single value serialization. + * The result returned is an array of bytes of Compact Theta sketch of Datasketches library, + * which should be deserialized to Compact sketch before using. + * + * See [[https://iceberg.apache.org/puffin-spec/]] for more information. + * + */ +case class ThetaSketchAgg( + child: Expression, + mutableAggBufferOffset: Int = 0, + inputAggBufferOffset: Int = 0) extends TypedImperativeAggregate[Sketch] with UnaryLike[Expression] { + + private lazy val icebergType = SparkSchemaUtil.convert(child.dataType) + + def this(colName: String) = { + this(col(colName).expr, 0, 0) + } + + override def dataType: DataType = BinaryType + + override def nullable: Boolean = false + + override def createAggregationBuffer(): Sketch = { + UpdateSketch.builder.setFamily(Family.ALPHA).build() + } + + override def update(buffer: Sketch, input: InternalRow): Sketch = { + val value = child.eval(input) + if (value != null) { + val icebergValue = toIcebergValue(value) + val byteBuffer = Conversions.toByteBuffer(icebergType, icebergValue) + buffer.asInstanceOf[UpdateSketch].update(byteBuffer) + } + buffer + } + + private def toIcebergValue(value: Any): Any = { + value match { + case s: UTF8String => s.toString + case d: Decimal => d.toJavaBigDecimal + case b: Array[Byte] => ByteBuffer.wrap(b) + case _ => value + } + } + + override def merge(buffer: Sketch, input: Sketch): Sketch = { + new SetOperationBuilder().buildUnion.union(buffer, input) + } + + override def eval(buffer: Sketch): Any = { + toBytes(buffer) + } + + override def serialize(buffer: Sketch): Array[Byte] = { + toBytes(buffer) + } + + override def deserialize(storageFormat: Array[Byte]): Sketch = { + CompactSketch.wrap(Memory.wrap(storageFormat)) + } + + override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate = { + copy(mutableAggBufferOffset = newMutableAggBufferOffset) + } + + override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate = { + copy(inputAggBufferOffset = newInputAggBufferOffset) + } + + override protected def withNewChildInternal(newChild: Expression): Expression = { + copy(child = newChild) + } + + private def toBytes(sketch: Sketch): Array[Byte] = { + val compactSketch = sketch.compact() + compactSketch.toByteArray + } +} diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestFileRewriteCoordinator.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestFileRewriteCoordinator.java index 3955d0395474..666634a06c02 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestFileRewriteCoordinator.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestFileRewriteCoordinator.java @@ -34,6 +34,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.source.SimpleRecord; +import org.apache.iceberg.util.DataFileSet; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; @@ -93,7 +94,7 @@ public void testBinPackRewrite() throws NoSuchTableException, IOException { Set rewrittenFiles = taskSetManager.fetchTasks(table, fileSetID).stream() .map(t -> t.asFileScanTask().file()) - .collect(Collectors.toSet()); + .collect(Collectors.toCollection(DataFileSet::create)); Set addedFiles = rewriteCoordinator.fetchNewFiles(table, fileSetID); table.newRewrite().rewriteFiles(rewrittenFiles, addedFiles).commit(); } @@ -165,7 +166,7 @@ public void testSortRewrite() throws NoSuchTableException, IOException { Set rewrittenFiles = taskSetManager.fetchTasks(table, fileSetID).stream() .map(t -> t.asFileScanTask().file()) - .collect(Collectors.toSet()); + .collect(Collectors.toCollection(DataFileSet::create)); Set addedFiles = rewriteCoordinator.fetchNewFiles(table, fileSetID); table.newRewrite().rewriteFiles(rewrittenFiles, addedFiles).commit(); } @@ -247,7 +248,7 @@ public void testCommitMultipleRewrites() throws NoSuchTableException, IOExceptio Set addedFiles = fileSetIDs.stream() .flatMap(fileSetID -> rewriteCoordinator.fetchNewFiles(table, fileSetID).stream()) - .collect(Collectors.toSet()); + .collect(Collectors.toCollection(DataFileSet::create)); table.newRewrite().rewriteFiles(rewrittenFiles, addedFiles).commit(); table.refresh(); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputeTableStatsAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputeTableStatsAction.java new file mode 100644 index 000000000000..7aa849d0bba8 --- /dev/null +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputeTableStatsAction.java @@ -0,0 +1,406 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.actions; + +import static org.apache.iceberg.spark.actions.NDVSketchUtil.APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatNoException; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.util.List; +import org.apache.iceberg.BlobMetadata; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.Files; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StatisticsFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.ComputeTableStats; +import org.apache.iceberg.data.FileHelpers; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.CatalogTestBase; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.spark.source.SimpleRecord; +import org.apache.iceberg.types.Types; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.parser.ParseException; +import org.apache.spark.sql.types.StructType; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; + +public class TestComputeTableStatsAction extends CatalogTestBase { + + private static final Types.StructType LEAF_STRUCT_TYPE = + Types.StructType.of( + optional(1, "leafLongCol", Types.LongType.get()), + optional(2, "leafDoubleCol", Types.DoubleType.get())); + + private static final Types.StructType NESTED_STRUCT_TYPE = + Types.StructType.of(required(3, "leafStructCol", LEAF_STRUCT_TYPE)); + + private static final Schema NESTED_SCHEMA = + new Schema(required(4, "nestedStructCol", NESTED_STRUCT_TYPE)); + + private static final Schema SCHEMA_WITH_NESTED_COLUMN = + new Schema( + required(4, "nestedStructCol", NESTED_STRUCT_TYPE), + required(5, "stringCol", Types.StringType.get())); + + @TestTemplate + public void testLoadingTableDirectly() { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + sql("INSERT into %s values(1, 'abcd')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result results = actions.computeTableStats(table).execute(); + StatisticsFile statisticsFile = results.statisticsFile(); + assertThat(statisticsFile.fileSizeInBytes()).isGreaterThan(0); + assertThat(statisticsFile.blobMetadata()).hasSize(2); + } + + @TestTemplate + public void testComputeTableStatsAction() throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + + // To create multiple splits on the mapper + table + .updateProperties() + .set("read.split.target-size", "100") + .set("write.parquet.row-group-size-bytes", "100") + .commit(); + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "c"), + new SimpleRecord(4, "d")); + spark.createDataset(records, Encoders.bean(SimpleRecord.class)).writeTo(tableName).append(); + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result results = + actions.computeTableStats(table).columns("id", "data").execute(); + assertThat(results).isNotNull(); + + List statisticsFiles = table.statisticsFiles(); + assertThat(statisticsFiles).hasSize(1); + + StatisticsFile statisticsFile = statisticsFiles.get(0); + assertThat(statisticsFile.fileSizeInBytes()).isGreaterThan(0); + assertThat(statisticsFile.blobMetadata()).hasSize(2); + + BlobMetadata blobMetadata = statisticsFile.blobMetadata().get(0); + assertThat(blobMetadata.properties()) + .containsEntry(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY, "4"); + } + + @TestTemplate + public void testComputeTableStatsActionWithoutExplicitColumns() + throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "c"), + new SimpleRecord(4, "d")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result results = actions.computeTableStats(table).execute(); + assertThat(results).isNotNull(); + + assertThat(table.statisticsFiles()).hasSize(1); + StatisticsFile statisticsFile = table.statisticsFiles().get(0); + assertThat(statisticsFile.fileSizeInBytes()).isGreaterThan(0); + assertThat(statisticsFile.blobMetadata()).hasSize(2); + assertThat(statisticsFile.blobMetadata().get(0).properties()) + .containsEntry(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY, "4"); + assertThat(statisticsFile.blobMetadata().get(1).properties()) + .containsEntry(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY, "4"); + } + + @TestTemplate + public void testComputeTableStatsForInvalidColumns() throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + // Append data to create snapshot + sql("INSERT into %s values(1, 'abcd')", tableName); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + assertThatThrownBy(() -> actions.computeTableStats(table).columns("id1").execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Can't find column id1 in table"); + } + + @TestTemplate + public void testComputeTableStatsWithNoSnapshots() throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result result = actions.computeTableStats(table).columns("id").execute(); + assertThat(result.statisticsFile()).isNull(); + } + + @TestTemplate + public void testComputeTableStatsWithNullValues() throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + List records = + Lists.newArrayList( + new SimpleRecord(1, null), + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "c"), + new SimpleRecord(4, "d")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result results = actions.computeTableStats(table).columns("data").execute(); + assertThat(results).isNotNull(); + + List statisticsFiles = table.statisticsFiles(); + assertThat(statisticsFiles).hasSize(1); + + StatisticsFile statisticsFile = statisticsFiles.get(0); + assertThat(statisticsFile.fileSizeInBytes()).isGreaterThan(0); + assertThat(statisticsFile.blobMetadata()).hasSize(1); + + assertThat(statisticsFile.blobMetadata().get(0).properties()) + .containsEntry(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY, "4"); + } + + @TestTemplate + public void testComputeTableStatsWithSnapshotHavingDifferentSchemas() + throws NoSuchTableException, ParseException { + SparkActions actions = SparkActions.get(); + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + // Append data to create snapshot + sql("INSERT into %s values(1, 'abcd')", tableName); + long snapshotId1 = Spark3Util.loadIcebergTable(spark, tableName).currentSnapshot().snapshotId(); + // Snapshot id not specified + Table table = Spark3Util.loadIcebergTable(spark, tableName); + + assertThatNoException() + .isThrownBy(() -> actions.computeTableStats(table).columns("data").execute()); + + sql("ALTER TABLE %s DROP COLUMN %s", tableName, "data"); + // Append data to create snapshot + sql("INSERT into %s values(1)", tableName); + table.refresh(); + long snapshotId2 = Spark3Util.loadIcebergTable(spark, tableName).currentSnapshot().snapshotId(); + + // Snapshot id specified + assertThatNoException() + .isThrownBy( + () -> actions.computeTableStats(table).snapshot(snapshotId1).columns("data").execute()); + + assertThatThrownBy( + () -> actions.computeTableStats(table).snapshot(snapshotId2).columns("data").execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Can't find column data in table"); + } + + @TestTemplate + public void testComputeTableStatsWhenSnapshotIdNotSpecified() + throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + // Append data to create snapshot + sql("INSERT into %s values(1, 'abcd')", tableName); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result results = actions.computeTableStats(table).columns("data").execute(); + + assertThat(results).isNotNull(); + + List statisticsFiles = table.statisticsFiles(); + assertThat(statisticsFiles).hasSize(1); + + StatisticsFile statisticsFile = statisticsFiles.get(0); + assertThat(statisticsFile.fileSizeInBytes()).isGreaterThan(0); + assertThat(statisticsFile.blobMetadata()).hasSize(1); + + assertThat(statisticsFile.blobMetadata().get(0).properties()) + .containsEntry(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY, "1"); + } + + @TestTemplate + public void testComputeTableStatsWithNestedSchema() + throws NoSuchTableException, ParseException, IOException { + List records = Lists.newArrayList(createNestedRecord()); + Table table = + validationCatalog.createTable( + tableIdent, + SCHEMA_WITH_NESTED_COLUMN, + PartitionSpec.unpartitioned(), + ImmutableMap.of()); + DataFile dataFile = FileHelpers.writeDataFile(table, Files.localOutput(temp.toFile()), records); + table.newAppend().appendFile(dataFile).commit(); + + Table tbl = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + actions.computeTableStats(tbl).execute(); + + tbl.refresh(); + List statisticsFiles = tbl.statisticsFiles(); + assertThat(statisticsFiles).hasSize(1); + StatisticsFile statisticsFile = statisticsFiles.get(0); + assertThat(statisticsFile.fileSizeInBytes()).isGreaterThan(0); + assertThat(statisticsFile.blobMetadata()).hasSize(1); + } + + @TestTemplate + public void testComputeTableStatsWithNoComputableColumns() throws IOException { + List records = Lists.newArrayList(createNestedRecord()); + Table table = + validationCatalog.createTable( + tableIdent, NESTED_SCHEMA, PartitionSpec.unpartitioned(), ImmutableMap.of()); + DataFile dataFile = FileHelpers.writeDataFile(table, Files.localOutput(temp.toFile()), records); + table.newAppend().appendFile(dataFile).commit(); + + table.refresh(); + SparkActions actions = SparkActions.get(); + assertThatThrownBy(() -> actions.computeTableStats(table).execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("No columns found to compute stats"); + } + + @TestTemplate + public void testComputeTableStatsOnByteColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("byte_col", "TINYINT"); + } + + @TestTemplate + public void testComputeTableStatsOnShortColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("short_col", "SMALLINT"); + } + + @TestTemplate + public void testComputeTableStatsOnIntColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("int_col", "INT"); + } + + @TestTemplate + public void testComputeTableStatsOnLongColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("long_col", "BIGINT"); + } + + @TestTemplate + public void testComputeTableStatsOnTimestampColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("timestamp_col", "TIMESTAMP"); + } + + @TestTemplate + public void testComputeTableStatsOnTimestampNtzColumn() + throws NoSuchTableException, ParseException { + testComputeTableStats("timestamp_col", "TIMESTAMP_NTZ"); + } + + @TestTemplate + public void testComputeTableStatsOnDateColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("date_col", "DATE"); + } + + @TestTemplate + public void testComputeTableStatsOnDecimalColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("decimal_col", "DECIMAL(20, 2)"); + } + + @TestTemplate + public void testComputeTableStatsOnBinaryColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("binary_col", "BINARY"); + } + + public void testComputeTableStats(String columnName, String type) + throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, %s %s) USING iceberg", tableName, columnName, type); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + + Dataset dataDF = randomDataDF(table.schema()); + append(tableName, dataDF); + + SparkActions actions = SparkActions.get(); + table.refresh(); + ComputeTableStats.Result results = + actions.computeTableStats(table).columns(columnName).execute(); + assertThat(results).isNotNull(); + + List statisticsFiles = table.statisticsFiles(); + assertThat(statisticsFiles).hasSize(1); + + StatisticsFile statisticsFile = statisticsFiles.get(0); + assertThat(statisticsFile.fileSizeInBytes()).isGreaterThan(0); + assertThat(statisticsFile.blobMetadata()).hasSize(1); + + assertThat(statisticsFile.blobMetadata().get(0).properties()) + .containsKey(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY); + } + + private GenericRecord createNestedRecord() { + GenericRecord record = GenericRecord.create(SCHEMA_WITH_NESTED_COLUMN); + GenericRecord nested = GenericRecord.create(NESTED_STRUCT_TYPE); + GenericRecord leaf = GenericRecord.create(LEAF_STRUCT_TYPE); + leaf.set(0, 0L); + leaf.set(1, 0.0); + nested.set(0, leaf); + record.set(0, nested); + record.set(1, "data"); + return record; + } + + private Dataset randomDataDF(Schema schema) { + Iterable rows = RandomData.generateSpark(schema, 10, 0); + JavaRDD rowRDD = sparkContext.parallelize(Lists.newArrayList(rows)); + StructType rowSparkType = SparkSchemaUtil.convert(schema); + return spark.internalCreateDataFrame(JavaRDD.toRDD(rowRDD), rowSparkType, false); + } + + private void append(String table, Dataset df) throws NoSuchTableException { + // fanout writes are enabled as write-time clustering is not supported without Spark extensions + df.coalesce(1).writeTo(table).option(SparkWriteOptions.FANOUT_ENABLED, "true").append(); + } + + @AfterEach + public void removeTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + } +} diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java index a0f1fba0434c..e2ebf7e8817c 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java @@ -186,6 +186,11 @@ public void before() { public void after() throws IOException { // Drop the hive table. spark.sql(String.format("DROP TABLE IF EXISTS %s", baseTableName)); + spark.sessionState().catalogManager().reset(); + spark.conf().unset("spark.sql.catalog.spark_catalog.type"); + spark.conf().unset("spark.sql.catalog.spark_catalog.default-namespace"); + spark.conf().unset("spark.sql.catalog.spark_catalog.parquet-enabled"); + spark.conf().unset("spark.sql.catalog.spark_catalog.cache-enabled"); } @TestTemplate @@ -746,6 +751,8 @@ public void testStructOfThreeLevelLists() throws Exception { @TestTemplate public void testTwoLevelList() throws IOException { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", true); String tableName = sourceName("testTwoLevelList"); @@ -829,6 +836,8 @@ public boolean accept(File dir, String name) { } private void threeLevelList(boolean useLegacyMode) throws Exception { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); String tableName = sourceName(String.format("threeLevelList_%s", useLegacyMode)); @@ -851,6 +860,8 @@ private void threeLevelList(boolean useLegacyMode) throws Exception { } private void threeLevelListWithNestedStruct(boolean useLegacyMode) throws Exception { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); String tableName = @@ -876,6 +887,8 @@ private void threeLevelListWithNestedStruct(boolean useLegacyMode) throws Except } private void threeLevelLists(boolean useLegacyMode) throws Exception { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); String tableName = sourceName(String.format("threeLevelLists_%s", useLegacyMode)); @@ -903,6 +916,8 @@ private void threeLevelLists(boolean useLegacyMode) throws Exception { } private void structOfThreeLevelLists(boolean useLegacyMode) throws Exception { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); String tableName = sourceName(String.format("structOfThreeLevelLists_%s", useLegacyMode)); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java new file mode 100644 index 000000000000..e15b2fb2174a --- /dev/null +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java @@ -0,0 +1,447 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.actions; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.nio.file.Path; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.actions.RemoveDanglingDeleteFiles; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.spark.TestBase; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Encoders; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import scala.Tuple2; + +public class TestRemoveDanglingDeleteAction extends TestBase { + + private static final HadoopTables TABLES = new HadoopTables(new Configuration()); + private static final Schema SCHEMA = + new Schema( + optional(1, "c1", Types.StringType.get()), + optional(2, "c2", Types.StringType.get()), + optional(3, "c3", Types.StringType.get())); + + private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build(); + + static final DataFile FILE_A = + DataFiles.builder(SPEC) + .withPath("/path/to/data-a.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=a") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_A2 = + DataFiles.builder(SPEC) + .withPath("/path/to/data-a.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=a") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_B = + DataFiles.builder(SPEC) + .withPath("/path/to/data-b.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=b") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_B2 = + DataFiles.builder(SPEC) + .withPath("/path/to/data-b.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=b") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_C = + DataFiles.builder(SPEC) + .withPath("/path/to/data-c.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=c") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_C2 = + DataFiles.builder(SPEC) + .withPath("/path/to/data-c.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=c") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_D = + DataFiles.builder(SPEC) + .withPath("/path/to/data-d.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=d") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_D2 = + DataFiles.builder(SPEC) + .withPath("/path/to/data-d.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=d") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_A_POS_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("/path/to/data-a-pos-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=a") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_A2_POS_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("/path/to/data-a2-pos-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=a") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_A_EQ_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofEqualityDeletes() + .withPath("/path/to/data-a-eq-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=a") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_A2_EQ_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofEqualityDeletes() + .withPath("/path/to/data-a2-eq-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=a") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_B_POS_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("/path/to/data-b-pos-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=b") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_B2_POS_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("/path/to/data-b2-pos-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=b") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_B_EQ_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofEqualityDeletes() + .withPath("/path/to/data-b-eq-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=b") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_B2_EQ_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofEqualityDeletes() + .withPath("/path/to/data-b2-eq-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=b") // easy way to set partition data for now + .withRecordCount(1) + .build(); + + static final DataFile FILE_UNPARTITIONED = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-unpartitioned.parquet") + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + static final DeleteFile FILE_UNPARTITIONED_POS_DELETE = + FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofEqualityDeletes() + .withPath("/path/to/data-unpartitioned-pos-deletes.parquet") + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + static final DeleteFile FILE_UNPARTITIONED_EQ_DELETE = + FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofEqualityDeletes() + .withPath("/path/to/data-unpartitioned-eq-deletes.parquet") + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + + @TempDir private Path temp; + + private String tableLocation = null; + private Table table; + + @BeforeEach + public void before() throws Exception { + File tableDir = temp.resolve("junit").toFile(); + this.tableLocation = tableDir.toURI().toString(); + } + + @AfterEach + public void after() { + TABLES.dropTable(tableLocation); + } + + private void setupPartitionedTable() { + this.table = + TABLES.create( + SCHEMA, SPEC, ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"), tableLocation); + } + + private void setupUnpartitionedTable() { + this.table = + TABLES.create( + SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"), + tableLocation); + } + + @Test + public void testPartitionedDeletesWithLesserSeqNo() { + setupPartitionedTable(); + + // Add Data Files + table.newAppend().appendFile(FILE_B).appendFile(FILE_C).appendFile(FILE_D).commit(); + + // Add Delete Files + table + .newRowDelta() + .addDeletes(FILE_A_POS_DELETES) + .addDeletes(FILE_A2_POS_DELETES) + .addDeletes(FILE_B_POS_DELETES) + .addDeletes(FILE_B2_POS_DELETES) + .addDeletes(FILE_A_EQ_DELETES) + .addDeletes(FILE_A2_EQ_DELETES) + .addDeletes(FILE_B_EQ_DELETES) + .addDeletes(FILE_B2_EQ_DELETES) + .commit(); + + // Add More Data Files + table + .newAppend() + .appendFile(FILE_A2) + .appendFile(FILE_B2) + .appendFile(FILE_C2) + .appendFile(FILE_D2) + .commit(); + + List> actual = + spark + .read() + .format("iceberg") + .load(tableLocation + "#entries") + .select("sequence_number", "data_file.file_path") + .sort("sequence_number", "data_file.file_path") + .as(Encoders.tuple(Encoders.LONG(), Encoders.STRING())) + .collectAsList(); + List> expected = + ImmutableList.of( + Tuple2.apply(1L, FILE_B.path().toString()), + Tuple2.apply(1L, FILE_C.path().toString()), + Tuple2.apply(1L, FILE_D.path().toString()), + Tuple2.apply(2L, FILE_A_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A2_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A2_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B2_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B2_POS_DELETES.path().toString()), + Tuple2.apply(3L, FILE_A2.path().toString()), + Tuple2.apply(3L, FILE_B2.path().toString()), + Tuple2.apply(3L, FILE_C2.path().toString()), + Tuple2.apply(3L, FILE_D2.path().toString())); + assertThat(actual).isEqualTo(expected); + + RemoveDanglingDeleteFiles.Result result = + SparkActions.get().removeDanglingDeleteFiles(table).execute(); + + // All Delete files of the FILE A partition should be removed + // because there are no data files in partition with a lesser sequence number + + Set removedDeleteFiles = + StreamSupport.stream(result.removedDeleteFiles().spliterator(), false) + .map(DeleteFile::path) + .collect(Collectors.toSet()); + assertThat(removedDeleteFiles) + .as("Expected 4 delete files removed") + .hasSize(4) + .containsExactlyInAnyOrder( + FILE_A_POS_DELETES.path(), + FILE_A2_POS_DELETES.path(), + FILE_A_EQ_DELETES.path(), + FILE_A2_EQ_DELETES.path()); + + List> actualAfter = + spark + .read() + .format("iceberg") + .load(tableLocation + "#entries") + .filter("status < 2") // live files + .select("sequence_number", "data_file.file_path") + .sort("sequence_number", "data_file.file_path") + .as(Encoders.tuple(Encoders.LONG(), Encoders.STRING())) + .collectAsList(); + List> expectedAfter = + ImmutableList.of( + Tuple2.apply(1L, FILE_B.path().toString()), + Tuple2.apply(1L, FILE_C.path().toString()), + Tuple2.apply(1L, FILE_D.path().toString()), + Tuple2.apply(2L, FILE_B_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B2_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B2_POS_DELETES.path().toString()), + Tuple2.apply(3L, FILE_A2.path().toString()), + Tuple2.apply(3L, FILE_B2.path().toString()), + Tuple2.apply(3L, FILE_C2.path().toString()), + Tuple2.apply(3L, FILE_D2.path().toString())); + assertThat(actualAfter).isEqualTo(expectedAfter); + } + + @Test + public void testPartitionedDeletesWithEqSeqNo() { + setupPartitionedTable(); + + // Add Data Files + table.newAppend().appendFile(FILE_A).appendFile(FILE_C).appendFile(FILE_D).commit(); + + // Add Data Files with EQ and POS deletes + table + .newRowDelta() + .addRows(FILE_A2) + .addRows(FILE_B2) + .addRows(FILE_C2) + .addRows(FILE_D2) + .addDeletes(FILE_A_POS_DELETES) + .addDeletes(FILE_A2_POS_DELETES) + .addDeletes(FILE_A_EQ_DELETES) + .addDeletes(FILE_A2_EQ_DELETES) + .addDeletes(FILE_B_POS_DELETES) + .addDeletes(FILE_B2_POS_DELETES) + .addDeletes(FILE_B_EQ_DELETES) + .addDeletes(FILE_B2_EQ_DELETES) + .commit(); + + List> actual = + spark + .read() + .format("iceberg") + .load(tableLocation + "#entries") + .select("sequence_number", "data_file.file_path") + .sort("sequence_number", "data_file.file_path") + .as(Encoders.tuple(Encoders.LONG(), Encoders.STRING())) + .collectAsList(); + List> expected = + ImmutableList.of( + Tuple2.apply(1L, FILE_A.path().toString()), + Tuple2.apply(1L, FILE_C.path().toString()), + Tuple2.apply(1L, FILE_D.path().toString()), + Tuple2.apply(2L, FILE_A_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A2.path().toString()), + Tuple2.apply(2L, FILE_A2_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A2_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B2.path().toString()), + Tuple2.apply(2L, FILE_B2_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B2_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_C2.path().toString()), + Tuple2.apply(2L, FILE_D2.path().toString())); + assertThat(actual).isEqualTo(expected); + + RemoveDanglingDeleteFiles.Result result = + SparkActions.get().removeDanglingDeleteFiles(table).execute(); + + // Eq Delete files of the FILE B partition should be removed + // because there are no data files in partition with a lesser sequence number + Set removedDeleteFiles = + StreamSupport.stream(result.removedDeleteFiles().spliterator(), false) + .map(DeleteFile::path) + .collect(Collectors.toSet()); + assertThat(removedDeleteFiles) + .as("Expected two delete files removed") + .hasSize(2) + .containsExactlyInAnyOrder(FILE_B_EQ_DELETES.path(), FILE_B2_EQ_DELETES.path()); + + List> actualAfter = + spark + .read() + .format("iceberg") + .load(tableLocation + "#entries") + .filter("status < 2") // live files + .select("sequence_number", "data_file.file_path") + .sort("sequence_number", "data_file.file_path") + .as(Encoders.tuple(Encoders.LONG(), Encoders.STRING())) + .collectAsList(); + List> expectedAfter = + ImmutableList.of( + Tuple2.apply(1L, FILE_A.path().toString()), + Tuple2.apply(1L, FILE_C.path().toString()), + Tuple2.apply(1L, FILE_D.path().toString()), + Tuple2.apply(2L, FILE_A_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A2.path().toString()), + Tuple2.apply(2L, FILE_A2_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A2_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B2.path().toString()), + Tuple2.apply(2L, FILE_B2_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_C2.path().toString()), + Tuple2.apply(2L, FILE_D2.path().toString())); + assertThat(actualAfter).isEqualTo(expectedAfter); + } + + @Test + public void testUnpartitionedTable() { + setupUnpartitionedTable(); + + table + .newRowDelta() + .addDeletes(FILE_UNPARTITIONED_POS_DELETE) + .addDeletes(FILE_UNPARTITIONED_EQ_DELETE) + .commit(); + table.newAppend().appendFile(FILE_UNPARTITIONED).commit(); + + RemoveDanglingDeleteFiles.Result result = + SparkActions.get().removeDanglingDeleteFiles(table).execute(); + assertThat(result.removedDeleteFiles()).as("No-op for unpartitioned tables").isEmpty(); + } +} diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java index b67ee87c7d3e..2de83f8b355c 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java @@ -24,6 +24,7 @@ import static org.apache.spark.sql.functions.current_date; import static org.apache.spark.sql.functions.date_add; import static org.apache.spark.sql.functions.expr; +import static org.apache.spark.sql.functions.min; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.mockito.ArgumentMatchers.any; @@ -56,6 +57,7 @@ import org.apache.iceberg.FileScanTask; import org.apache.iceberg.MetadataTableType; import org.apache.iceberg.PartitionData; +import org.apache.iceberg.PartitionKey; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.RewriteJobOrder; import org.apache.iceberg.RowDelta; @@ -73,7 +75,9 @@ import org.apache.iceberg.actions.SizeBasedDataRewriter; import org.apache.iceberg.actions.SizeBasedFileRewriter; import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; +import org.apache.iceberg.deletes.EqualityDeleteWriter; import org.apache.iceberg.deletes.PositionDelete; import org.apache.iceberg.deletes.PositionDeleteWriter; import org.apache.iceberg.encryption.EncryptedFiles; @@ -86,6 +90,7 @@ import org.apache.iceberg.hadoop.HadoopTables; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.OutputFileFactory; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; @@ -105,9 +110,11 @@ import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.NestedField; +import org.apache.iceberg.util.ArrayUtil; import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.StructLikeMap; import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; import org.apache.spark.sql.internal.SQLConf; import org.junit.jupiter.api.BeforeAll; @@ -128,6 +135,8 @@ public class TestRewriteDataFilesAction extends TestBase { optional(2, "c2", Types.StringType.get()), optional(3, "c3", Types.StringType.get())); + private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build(); + @TempDir private Path temp; private final FileRewriteCoordinator coordinator = FileRewriteCoordinator.get(); @@ -336,6 +345,125 @@ public void testBinPackWithDeletes() { assertThat(actualRecords).as("7 rows are removed").hasSize(total - 7); } + @Test + public void testRemoveDangledEqualityDeletesPartitionEvolution() { + Table table = + TABLES.create( + SCHEMA, + SPEC, + Collections.singletonMap(TableProperties.FORMAT_VERSION, "2"), + tableLocation); + + // data seq = 1, write 4 files in 2 partitions + List records1 = + Lists.newArrayList( + new ThreeColumnRecord(1, null, "AAAA"), new ThreeColumnRecord(1, "BBBBBBBBBB", "BBBB")); + writeRecords(records1); + List records2 = + Lists.newArrayList( + new ThreeColumnRecord(0, "CCCCCCCCCC", "CCCC"), + new ThreeColumnRecord(0, "DDDDDDDDDD", "DDDD")); + writeRecords(records2); + table.refresh(); + shouldHaveFiles(table, 4); + + // data seq = 2 & 3, write 2 equality deletes in both partitions + writeEqDeleteRecord(table, "c1", 1, "c3", "AAAA"); + writeEqDeleteRecord(table, "c1", 2, "c3", "CCCC"); + table.refresh(); + Set existingDeletes = TestHelpers.deleteFiles(table); + assertThat(existingDeletes) + .as("Only one equality delete c1=1 is used in query planning") + .hasSize(1); + + // partition evolution + table.refresh(); + table.updateSpec().addField(Expressions.ref("c3")).commit(); + + // data seq = 4, write 2 new data files in both partitions for evolved spec + List records3 = + Lists.newArrayList( + new ThreeColumnRecord(1, "A", "CCCC"), new ThreeColumnRecord(2, "D", "DDDD")); + writeRecords(records3); + + List originalData = currentData(); + + RewriteDataFiles.Result result = + basicRewrite(table) + .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .filter(Expressions.equal("c1", 1)) + .option(RewriteDataFiles.REMOVE_DANGLING_DELETES, "true") + .execute(); + + existingDeletes = TestHelpers.deleteFiles(table); + assertThat(existingDeletes).as("Shall pruned dangling deletes after rewrite").hasSize(0); + + assertThat(result) + .extracting( + Result::addedDataFilesCount, + Result::rewrittenDataFilesCount, + Result::removedDeleteFilesCount) + .as("Should compact 3 data files into 2 and remove both dangled equality delete file") + .containsExactly(2, 3, 2); + shouldHaveMinSequenceNumberInPartition(table, "data_file.partition.c1 == 1", 5); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 7); + shouldHaveFiles(table, 5); + } + + @Test + public void testRemoveDangledPositionDeletesPartitionEvolution() { + Table table = + TABLES.create( + SCHEMA, + SPEC, + Collections.singletonMap(TableProperties.FORMAT_VERSION, "2"), + tableLocation); + + // data seq = 1, write 4 files in 2 partitions + writeRecords(2, 2, 2); + List dataFilesBefore = TestHelpers.dataFiles(table, null); + shouldHaveFiles(table, 4); + + // data seq = 2, write 1 position deletes in c1=1 + table + .newRowDelta() + .addDeletes(writePosDeletesToFile(table, dataFilesBefore.get(3), 1).get(0)) + .commit(); + + // partition evolution + table.updateSpec().addField(Expressions.ref("c3")).commit(); + + // data seq = 3, write 1 new data files in c1=1 for evolved spec + writeRecords(1, 1, 1); + shouldHaveFiles(table, 5); + List expectedRecords = currentData(); + + Result result = + actions() + .rewriteDataFiles(table) + .filter(Expressions.equal("c1", 1)) + .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(RewriteDataFiles.REMOVE_DANGLING_DELETES, "true") + .execute(); + + assertThat(result) + .extracting( + Result::addedDataFilesCount, + Result::rewrittenDataFilesCount, + Result::removedDeleteFilesCount) + .as("Should rewrite 2 data files into 1 and remove 1 dangled position delete file") + .containsExactly(1, 2, 1); + shouldHaveMinSequenceNumberInPartition(table, "data_file.partition.c1 == 1", 3); + + shouldHaveSnapshots(table, 5); + assertThat(table.currentSnapshot().summary().get("total-position-deletes")).isEqualTo("0"); + assertEquals("Rows must match", expectedRecords, currentData()); + } + @Test public void testBinPackWithDeleteAllData() { Map options = Maps.newHashMap(); @@ -1697,6 +1825,21 @@ protected void shouldHaveFiles(Table table, int numExpected) { assertThat(numFiles).as("Did not have the expected number of files").isEqualTo(numExpected); } + protected long shouldHaveMinSequenceNumberInPartition( + Table table, String partitionFilter, long expected) { + long actual = + SparkTableUtil.loadMetadataTable(spark, table, MetadataTableType.ENTRIES) + .filter("status != 2") + .filter(partitionFilter) + .select("sequence_number") + .agg(min("sequence_number")) + .as(Encoders.LONG()) + .collectAsList() + .get(0); + assertThat(actual).as("Did not have the expected min sequence number").isEqualTo(expected); + return actual; + } + protected void shouldHaveSnapshots(Table table, int expectedSnapshots) { table.refresh(); int actualSnapshots = Iterables.size(table.snapshots()); @@ -1893,6 +2036,11 @@ protected int averageFileSize(Table table) { .getAsDouble(); } + private void writeRecords(List records) { + Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class); + writeDF(df); + } + private void writeRecords(int files, int numRecords) { writeRecords(files, numRecords, 0); } @@ -1946,7 +2094,10 @@ private List writePosDeletes( table .io() .newOutputFile( - table.locationProvider().newDataLocation(UUID.randomUUID().toString())); + table + .locationProvider() + .newDataLocation( + FileFormat.PARQUET.addExtension(UUID.randomUUID().toString()))); EncryptedOutputFile encryptedOutputFile = EncryptedFiles.encryptedOutput(outputFile, EncryptionKeyMetadata.EMPTY); @@ -1972,6 +2123,68 @@ private List writePosDeletes( return results; } + private void writeEqDeleteRecord( + Table table, String partCol, Object partVal, String delCol, Object delVal) { + List equalityFieldIds = Lists.newArrayList(table.schema().findField(delCol).fieldId()); + Schema eqDeleteRowSchema = table.schema().select(delCol); + Record partitionRecord = + GenericRecord.create(table.schema().select(partCol)) + .copy(ImmutableMap.of(partCol, partVal)); + Record record = GenericRecord.create(eqDeleteRowSchema).copy(ImmutableMap.of(delCol, delVal)); + writeEqDeleteRecord(table, equalityFieldIds, partitionRecord, eqDeleteRowSchema, record); + } + + private void writeEqDeleteRecord( + Table table, + List equalityFieldIds, + Record partitionRecord, + Schema eqDeleteRowSchema, + Record deleteRecord) { + OutputFileFactory fileFactory = + OutputFileFactory.builderFor(table, 1, 1).format(FileFormat.PARQUET).build(); + GenericAppenderFactory appenderFactory = + new GenericAppenderFactory( + table.schema(), + table.spec(), + ArrayUtil.toIntArray(equalityFieldIds), + eqDeleteRowSchema, + null); + + EncryptedOutputFile file = + createEncryptedOutputFile(createPartitionKey(table, partitionRecord), fileFactory); + + EqualityDeleteWriter eqDeleteWriter = + appenderFactory.newEqDeleteWriter( + file, FileFormat.PARQUET, createPartitionKey(table, partitionRecord)); + + try (EqualityDeleteWriter clsEqDeleteWriter = eqDeleteWriter) { + clsEqDeleteWriter.write(deleteRecord); + } catch (Exception e) { + throw new RuntimeException(e); + } + table.newRowDelta().addDeletes(eqDeleteWriter.toDeleteFile()).commit(); + } + + private PartitionKey createPartitionKey(Table table, Record record) { + if (table.spec().isUnpartitioned()) { + return null; + } + + PartitionKey partitionKey = new PartitionKey(table.spec(), table.schema()); + partitionKey.partition(record); + + return partitionKey; + } + + private EncryptedOutputFile createEncryptedOutputFile( + PartitionKey partition, OutputFileFactory fileFactory) { + if (partition == null) { + return fileFactory.newOutputFile(); + } else { + return fileFactory.newOutputFile(partition); + } + } + private SparkActions actions() { return SparkActions.get(); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java index 37b6cd86fb92..8547f9753f5e 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java @@ -862,6 +862,7 @@ private void writePosDeletesForFiles( files.stream().collect(Collectors.groupingBy(ContentFile::partition)); List deleteFiles = Lists.newArrayListWithCapacity(deleteFilesPerPartition * filesByPartition.size()); + String suffix = String.format(".%s", FileFormat.PARQUET.name().toLowerCase()); for (Map.Entry> filesByPartitionEntry : filesByPartition.entrySet()) { @@ -886,7 +887,7 @@ private void writePosDeletesForFiles( if (counter == deleteFileSize) { // Dump to file and reset variables OutputFile output = - Files.localOutput(File.createTempFile("junit", null, temp.toFile())); + Files.localOutput(File.createTempFile("junit", suffix, temp.toFile())); deleteFiles.add(FileHelpers.writeDeleteFile(table, output, partition, deletes).first()); counter = 0; deletes.clear(); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java index c73ef630ac48..d64ca588f202 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java @@ -54,11 +54,11 @@ import org.apache.iceberg.TableScan; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.relocated.com.google.common.collect.Streams; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.DeleteFileSet; import org.apache.orc.storage.serde2.io.DateWritable; import org.apache.spark.sql.Column; import org.apache.spark.sql.Dataset; @@ -116,6 +116,21 @@ public static void assertEqualsBatch( } } + public static void assertEqualsBatchWithRows( + Types.StructType struct, Iterator expected, ColumnarBatch batch) { + for (int rowId = 0; rowId < batch.numRows(); rowId++) { + List fields = struct.fields(); + InternalRow row = batch.getRow(rowId); + Row expectedRow = expected.next(); + for (int i = 0; i < fields.size(); i += 1) { + Type fieldType = fields.get(i).type(); + Object expectedValue = expectedRow.get(i); + Object actualValue = row.isNullAt(i) ? null : row.get(i, convert(fieldType)); + assertEqualsUnsafe(fieldType, expectedValue, actualValue); + } + } + } + private static void assertEqualsSafe(Types.ListType list, Collection expected, List actual) { Type elementType = list.elementType(); List expectedElements = Lists.newArrayList(expected); @@ -787,7 +802,7 @@ public static List dataFiles(Table table, String branch) { } public static Set deleteFiles(Table table) { - Set deleteFiles = Sets.newHashSet(); + DeleteFileSet deleteFiles = DeleteFileSet.create(); for (FileScanTask task : table.newScan().planFiles()) { deleteFiles.addAll(task.deletes()); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroEnums.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroEnums.java index 11e60187fdc3..0dc8b48b2317 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroEnums.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroEnums.java @@ -79,7 +79,7 @@ public void writeAndValidateEnums() throws IOException { List rows; try (AvroIterable reader = Avro.read(Files.localInput(testFile)) - .createReaderFunc(SparkAvroReader::new) + .createResolvingReader(SparkPlannedAvroReader::create) .project(schema) .build()) { rows = Lists.newArrayList(reader); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroReader.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroReader.java index 3e5088258a49..7f9bcbacf298 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroReader.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroReader.java @@ -51,7 +51,7 @@ protected void writeAndValidate(Schema schema) throws IOException { List rows; try (AvroIterable reader = Avro.read(Files.localInput(testFile)) - .createReaderFunc(SparkAvroReader::new) + .createResolvingReader(SparkPlannedAvroReader::create) .project(schema) .build()) { rows = Lists.newArrayList(reader); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetDictionaryEncodedVectorizedReads.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetDictionaryEncodedVectorizedReads.java index eeed9d1a03ce..bc4e722bc869 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetDictionaryEncodedVectorizedReads.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetDictionaryEncodedVectorizedReads.java @@ -18,13 +18,21 @@ */ package org.apache.iceberg.spark.data.parquet.vectorized; +import static org.apache.iceberg.TableProperties.PARQUET_DICT_SIZE_BYTES; +import static org.apache.iceberg.TableProperties.PARQUET_PAGE_ROW_LIMIT; import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES_DEFAULT; import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Iterator; +import java.util.List; import org.apache.avro.generic.GenericData; +import org.apache.iceberg.Files; import org.apache.iceberg.Schema; +import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.relocated.com.google.common.base.Function; @@ -33,11 +41,35 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.spark.data.TestHelpers; +import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.vectorized.ColumnarBatch; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; public class TestParquetDictionaryEncodedVectorizedReads extends TestParquetVectorizedReads { + protected static SparkSession spark = null; + + @BeforeAll + public static void startSpark() { + spark = SparkSession.builder().master("local[2]").getOrCreate(); + } + + @AfterAll + public static void stopSpark() { + if (spark != null) { + spark.stop(); + spark = null; + } + } + @Override Iterable generateData( Schema schema, @@ -93,4 +125,64 @@ public void testMixedDictionaryNonDictionaryReads() throws IOException { true, BATCH_SIZE); } + + @Test + public void testBinaryNotAllPagesDictionaryEncoded() throws IOException { + Schema schema = new Schema(Types.NestedField.required(1, "bytes", Types.BinaryType.get())); + File parquetFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(parquetFile.delete()).as("Delete should succeed").isTrue(); + + Iterable records = RandomData.generateFallbackData(schema, 500, 0L, 100); + try (FileAppender writer = + Parquet.write(Files.localOutput(parquetFile)) + .schema(schema) + .set(PARQUET_DICT_SIZE_BYTES, "4096") + .set(PARQUET_PAGE_ROW_LIMIT, "100") + .build()) { + writer.addAll(records); + } + + // After the above, parquetFile contains one column chunk of binary data in five pages, + // the first two RLE dictionary encoded, and the remaining three plain encoded. + assertRecordsMatch(schema, 500, records, parquetFile, true, BATCH_SIZE); + } + + /** + * decimal_dict_and_plain_encoding.parquet contains one column chunk of decimal(38, 0) data in two + * pages, one RLE dictionary encoded and one plain encoded, each with 200 rows. + */ + @Test + public void testDecimalNotAllPagesDictionaryEncoded() throws Exception { + Schema schema = new Schema(Types.NestedField.required(1, "id", Types.DecimalType.of(38, 0))); + Path path = + Paths.get( + getClass() + .getClassLoader() + .getResource("decimal_dict_and_plain_encoding.parquet") + .toURI()); + + Dataset df = spark.read().parquet(path.toString()); + List expected = df.collectAsList(); + long expectedSize = df.count(); + + Parquet.ReadBuilder readBuilder = + Parquet.read(Files.localInput(path.toFile())) + .project(schema) + .createBatchedReaderFunc( + type -> + VectorizedSparkParquetReaders.buildReader( + schema, type, ImmutableMap.of(), null)); + + try (CloseableIterable batchReader = readBuilder.build()) { + Iterator expectedIter = expected.iterator(); + Iterator batches = batchReader.iterator(); + int numRowsRead = 0; + while (batches.hasNext()) { + ColumnarBatch batch = batches.next(); + numRowsRead += batch.numRows(); + TestHelpers.assertEqualsBatchWithRows(schema.asStruct(), expectedIter, batch); + } + assertThat(numRowsRead).isEqualTo(expectedSize); + } + } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java index f569446f772b..217b05b46a7c 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java @@ -78,6 +78,7 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.io.TempDir; @@ -146,6 +147,13 @@ public static void startSpark() { TestCompressionSettings.spark = SparkSession.builder().master("local[2]").getOrCreate(); } + @BeforeEach + public void resetSpecificConfigurations() { + spark.conf().unset(COMPRESSION_CODEC); + spark.conf().unset(COMPRESSION_LEVEL); + spark.conf().unset(COMPRESSION_STRATEGY); + } + @AfterEach public void afterEach() { spark.sql(String.format("DROP TABLE IF EXISTS %s", TABLE_NAME)); @@ -191,6 +199,8 @@ public void testWriteDataWithDifferentSetting() throws Exception { spark.conf().set(entry.getKey(), entry.getValue()); } + assertSparkConf(); + df.select("id", "data") .writeTo(TABLE_NAME) .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) @@ -253,4 +263,13 @@ private String getCompressionType(InputFile inputFile) throws Exception { return fileReader.getMetaString(DataFileConstants.CODEC); } } + + private void assertSparkConf() { + String[] propertiesToCheck = {COMPRESSION_CODEC, COMPRESSION_LEVEL, COMPRESSION_STRATEGY}; + for (String prop : propertiesToCheck) { + String expected = properties.getOrDefault(prop, null); + String actual = spark.conf().get(prop, null); + assertThat(actual).isEqualToIgnoringCase(expected); + } + } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java index ffdaa8b477cb..1c87886241bf 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java @@ -209,4 +209,39 @@ public void testWriteWithCaseSensitiveOption() throws NoSuchTableException, Pars fields = Spark3Util.loadIcebergTable(sparkSession, tableName).schema().asStruct().fields(); assertThat(fields).hasSize(4); } + + @TestTemplate + public void testMergeSchemaSparkConfiguration() throws Exception { + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s'='true')", + tableName, TableProperties.SPARK_WRITE_ACCEPT_ANY_SCHEMA); + Dataset twoColDF = + jsonToDF( + "id bigint, data string", + "{ \"id\": 1, \"data\": \"a\" }", + "{ \"id\": 2, \"data\": \"b\" }"); + + twoColDF.writeTo(tableName).append(); + + assertEquals( + "Should have initial 2-column rows", + ImmutableList.of(row(1L, "a"), row(2L, "b")), + sql("select * from %s order by id", tableName)); + spark.conf().set("spark.sql.iceberg.merge-schema", "true"); + Dataset threeColDF = + jsonToDF( + "id bigint, data string, salary float", + "{ \"id\": 3, \"data\": \"c\", \"salary\": 120000.34 }", + "{ \"id\": 4, \"data\": \"d\", \"salary\": 140000.56 }"); + + threeColDF.writeTo(tableName).append(); + assertEquals( + "Should have 3-column rows", + ImmutableList.of( + row(1L, "a", null), + row(2L, "b", null), + row(3L, "c", 120000.34F), + row(4L, "d", 140000.56F)), + sql("select * from %s order by id", tableName)); + } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java index 336ee5a8d2ea..01c36b824ea6 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java @@ -28,6 +28,7 @@ import java.io.File; import java.io.IOException; import java.net.URI; +import java.nio.file.NoSuchFileException; import java.util.Arrays; import java.util.Collection; import java.util.Iterator; @@ -35,6 +36,7 @@ import java.util.Map; import java.util.Random; import org.apache.avro.generic.GenericData.Record; +import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.Files; import org.apache.iceberg.Parameter; @@ -56,7 +58,7 @@ import org.apache.iceberg.spark.SparkWriteOptions; import org.apache.iceberg.spark.data.ParameterizedAvroDataTest; import org.apache.iceberg.spark.data.RandomData; -import org.apache.iceberg.spark.data.SparkAvroReader; +import org.apache.iceberg.spark.data.SparkPlannedAvroReader; import org.apache.iceberg.types.Types; import org.apache.spark.SparkException; import org.apache.spark.TaskContext; @@ -259,7 +261,7 @@ private Dataset createDataset(Iterable records, Schema schema) thro List rows = Lists.newArrayList(); try (AvroIterable reader = Avro.read(Files.localInput(testFile)) - .createReaderFunc(SparkAvroReader::new) + .createResolvingReader(SparkPlannedAvroReader::create) .project(schema) .build()) { @@ -419,5 +421,13 @@ public void testFaultToleranceOnWrite() throws IOException { assertThat(snapshotBeforeFailingWrite).isEqualTo(snapshotAfterFailingWrite); assertThat(resultBeforeFailingWrite).isEqualTo(resultAfterFailingWrite); + + while (location.exists()) { + try { + FileUtils.deleteDirectory(location); + } catch (NoSuchFileException e) { + // ignore NoSuchFileException when a file is already deleted + } + } } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index d37d6a861690..29216150d362 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -2044,8 +2044,13 @@ public void testFilesTablePartitionId() { .save(loadLocation(tableIdentifier)); List actual = - spark.read().format("iceberg").load(loadLocation(tableIdentifier, "files")) - .sort(DataFile.SPEC_ID.name()).collectAsList().stream() + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "files")) + .sort(DataFile.SPEC_ID.name()) + .collectAsList() + .stream() .map(r -> (Integer) r.getAs(DataFile.SPEC_ID.name())) .collect(Collectors.toList()); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java index a6573171aa6a..ca934772f6af 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java @@ -72,6 +72,7 @@ import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.CharSequenceSet; +import org.apache.iceberg.util.DeleteFileSet; import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.StructLikeSet; import org.apache.spark.sql.AnalysisException; @@ -1576,7 +1577,7 @@ private void commit( Set rewrittenFiles = ScanTaskSetManager.get().fetchTasks(posDeletesTable, fileSetID).stream() .map(t -> ((PositionDeletesScanTask) t).file()) - .collect(Collectors.toSet()); + .collect(Collectors.toCollection(DeleteFileSet::create)); Set addedFiles = rewriteCoordinator.fetchNewFiles(posDeletesTable, fileSetID); // Assert new files and old files are equal in number but different in paths diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java index 4643836542aa..29c2d4b39a1e 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java @@ -37,6 +37,7 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.Files; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Parameter; @@ -99,8 +100,6 @@ public class TestSparkReaderDeletes extends DeleteReadTests { protected static SparkSession spark = null; protected static HiveCatalog catalog = null; - @Parameter private String format; - @Parameter(index = 1) private boolean vectorized; @@ -110,10 +109,10 @@ public class TestSparkReaderDeletes extends DeleteReadTests { @Parameters(name = "format = {0}, vectorized = {1}, planningMode = {2}") public static Object[][] parameters() { return new Object[][] { - new Object[] {"parquet", false, PlanningMode.DISTRIBUTED}, - new Object[] {"parquet", true, PlanningMode.LOCAL}, - new Object[] {"orc", false, PlanningMode.DISTRIBUTED}, - new Object[] {"avro", false, PlanningMode.LOCAL} + new Object[] {FileFormat.PARQUET, false, PlanningMode.DISTRIBUTED}, + new Object[] {FileFormat.PARQUET, true, PlanningMode.LOCAL}, + new Object[] {FileFormat.ORC, false, PlanningMode.DISTRIBUTED}, + new Object[] {FileFormat.AVRO, false, PlanningMode.LOCAL} }; } @@ -169,17 +168,17 @@ protected Table createTable(String name, Schema schema, PartitionSpec spec) { ops.commit(meta, meta.upgradeToFormatVersion(2)); table .updateProperties() - .set(TableProperties.DEFAULT_FILE_FORMAT, format) + .set(TableProperties.DEFAULT_FILE_FORMAT, format.name()) .set(TableProperties.DATA_PLANNING_MODE, planningMode.modeName()) .set(TableProperties.DELETE_PLANNING_MODE, planningMode.modeName()) .commit(); - if (format.equals("parquet") || format.equals("orc")) { + if (format.equals(FileFormat.PARQUET) || format.equals(FileFormat.ORC)) { String vectorizationEnabled = - format.equals("parquet") + format.equals(FileFormat.PARQUET) ? TableProperties.PARQUET_VECTORIZATION_ENABLED : TableProperties.ORC_VECTORIZATION_ENABLED; String batchSize = - format.equals("parquet") + format.equals(FileFormat.PARQUET) ? TableProperties.PARQUET_BATCH_SIZE : TableProperties.ORC_BATCH_SIZE; table.updateProperties().set(vectorizationEnabled, String.valueOf(vectorized)).commit(); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java index 7d5475ff919e..af144fe4bf3d 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java @@ -82,6 +82,8 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestSparkScan extends TestBaseWithCatalog { + private static final String DUMMY_BLOB_TYPE = "sum-data-size-bytes-v1"; + @Parameter(index = 3) private String format; @@ -178,6 +180,59 @@ public void testTableWithoutColStats() throws NoSuchTableException { reportColStatsEnabled, () -> checkColStatisticsReported(scan, 4L, Maps.newHashMap())); } + @TestTemplate + public void testTableWithoutApacheDatasketchColStat() throws NoSuchTableException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "a"), + new SimpleRecord(4, "b")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + Table table = validationCatalog.loadTable(tableIdent); + long snapshotId = table.currentSnapshot().snapshotId(); + + SparkScanBuilder scanBuilder = + new SparkScanBuilder(spark, table, CaseInsensitiveStringMap.empty()); + SparkScan scan = (SparkScan) scanBuilder.build(); + + Map reportColStatsDisabled = + ImmutableMap.of( + SQLConf.CBO_ENABLED().key(), "true", SparkSQLProperties.REPORT_COLUMN_STATS, "false"); + + Map reportColStatsEnabled = + ImmutableMap.of(SQLConf.CBO_ENABLED().key(), "true"); + + GenericStatisticsFile statisticsFile = + new GenericStatisticsFile( + snapshotId, + "/test/statistics/file.puffin", + 100, + 42, + ImmutableList.of( + new GenericBlobMetadata( + DUMMY_BLOB_TYPE, + snapshotId, + 1, + ImmutableList.of(1), + ImmutableMap.of("data_size", "4")))); + + table.updateStatistics().setStatistics(snapshotId, statisticsFile).commit(); + + checkColStatisticsNotReported(scan, 4L); + withSQLConf(reportColStatsDisabled, () -> checkColStatisticsNotReported(scan, 4L)); + // The expected col NDVs are nulls + withSQLConf( + reportColStatsEnabled, () -> checkColStatisticsReported(scan, 4L, Maps.newHashMap())); + } + @TestTemplate public void testTableWithOneColStats() throws NoSuchTableException { sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); @@ -232,6 +287,67 @@ public void testTableWithOneColStats() throws NoSuchTableException { withSQLConf(reportColStatsEnabled, () -> checkColStatisticsReported(scan, 4L, expectedOneNDV)); } + @TestTemplate + public void testTableWithOneApacheDatasketchColStatAndOneDifferentColStat() + throws NoSuchTableException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "a"), + new SimpleRecord(4, "b")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + Table table = validationCatalog.loadTable(tableIdent); + long snapshotId = table.currentSnapshot().snapshotId(); + + SparkScanBuilder scanBuilder = + new SparkScanBuilder(spark, table, CaseInsensitiveStringMap.empty()); + SparkScan scan = (SparkScan) scanBuilder.build(); + + Map reportColStatsDisabled = + ImmutableMap.of( + SQLConf.CBO_ENABLED().key(), "true", SparkSQLProperties.REPORT_COLUMN_STATS, "false"); + + Map reportColStatsEnabled = + ImmutableMap.of(SQLConf.CBO_ENABLED().key(), "true"); + + GenericStatisticsFile statisticsFile = + new GenericStatisticsFile( + snapshotId, + "/test/statistics/file.puffin", + 100, + 42, + ImmutableList.of( + new GenericBlobMetadata( + APACHE_DATASKETCHES_THETA_V1, + snapshotId, + 1, + ImmutableList.of(1), + ImmutableMap.of("ndv", "4")), + new GenericBlobMetadata( + DUMMY_BLOB_TYPE, + snapshotId, + 1, + ImmutableList.of(1), + ImmutableMap.of("data_size", "2")))); + + table.updateStatistics().setStatistics(snapshotId, statisticsFile).commit(); + + checkColStatisticsNotReported(scan, 4L); + withSQLConf(reportColStatsDisabled, () -> checkColStatisticsNotReported(scan, 4L)); + + Map expectedOneNDV = Maps.newHashMap(); + expectedOneNDV.put("id", 4L); + withSQLConf(reportColStatsEnabled, () -> checkColStatisticsReported(scan, 4L, expectedOneNDV)); + } + @TestTemplate public void testTableWithTwoColStats() throws NoSuchTableException { sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); @@ -911,9 +1027,15 @@ private void checkColStatisticsReported( assertThat(stats.numRows().getAsLong()).isEqualTo(expectedRowCount); Map columnStats = stats.columnStats(); - for (Map.Entry entry : expectedNDVs.entrySet()) { - assertThat(columnStats.get(FieldReference.column(entry.getKey())).distinctCount().getAsLong()) - .isEqualTo(entry.getValue()); + if (expectedNDVs.isEmpty()) { + assertThat(columnStats.values().stream().allMatch(value -> value.distinctCount().isEmpty())) + .isTrue(); + } else { + for (Map.Entry entry : expectedNDVs.entrySet()) { + assertThat( + columnStats.get(FieldReference.column(entry.getKey())).distinctCount().getAsLong()) + .isEqualTo(entry.getValue()); + } } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java index 7e9bdeec8af0..6e09252704a1 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java @@ -54,6 +54,8 @@ public static void startMetastoreAndSpark() { metastore.start(); TestBase.hiveConf = metastore.hiveConf(); + TestBase.spark.close(); + TestBase.spark = SparkSession.builder() .master("local[2]") diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWritesToBranch.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWritesToBranch.java index a0c664b03b83..3df5e9cdf5da 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWritesToBranch.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWritesToBranch.java @@ -22,6 +22,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; public class TestUnpartitionedWritesToBranch extends UnpartitionedWritesTestBase { @@ -29,6 +30,7 @@ public class TestUnpartitionedWritesToBranch extends UnpartitionedWritesTestBase private static final String BRANCH = "test"; @Override + @BeforeEach public void createTables() { super.createTables(); Table table = validationCatalog.loadTable(tableIdent); diff --git a/spark/v3.5/spark/src/test/resources/decimal_dict_and_plain_encoding.parquet b/spark/v3.5/spark/src/test/resources/decimal_dict_and_plain_encoding.parquet new file mode 100644 index 000000000000..48b3bd1bf24f Binary files /dev/null and b/spark/v3.5/spark/src/test/resources/decimal_dict_and_plain_encoding.parquet differ