diff --git a/.asf.yaml b/.asf.yaml index bf45b54a1787..d93e94ebf65f 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -39,6 +39,8 @@ github: required_approving_review_count: 1 required_linear_history: true + + del_branch_on_merge: true features: wiki: true diff --git a/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml b/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml index a04263e8c943..61b21cb49d38 100644 --- a/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml +++ b/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml @@ -28,7 +28,9 @@ body: description: What Apache Iceberg version are you using? multiple: false options: - - "1.6.1 (latest release)" + - "1.7.1 (latest release)" + - "1.7.0" + - "1.6.1" - "1.6.0" - "1.5.2" - "1.5.1" diff --git a/.github/labeler.yml b/.github/labeler.yml index d11c68264cb5..6afc3141ee31 100644 --- a/.github/labeler.yml +++ b/.github/labeler.yml @@ -130,12 +130,6 @@ MR: 'mr/**/*' ] -PIG: - - changed-files: - - any-glob-to-any-file: [ - 'pig/**/*' - ] - AWS: - changed-files: - any-glob-to-any-file: [ diff --git a/.github/workflows/delta-conversion-ci.yml b/.github/workflows/delta-conversion-ci.yml index 9326d9d533fd..7b8fcfc992b8 100644 --- a/.github/workflows/delta-conversion-ci.yml +++ b/.github/workflows/delta-conversion-ci.yml @@ -37,7 +37,7 @@ on: - '.github/workflows/jmh-benchmarks-ci.yml' - '.github/workflows/kafka-connect-ci.yml' - '.github/workflows/labeler.yml' - - '.github/workflows/licence-check.yml' + - '.github/workflows/license-check.yml' - '.github/workflows/open-api.yml' - '.github/workflows/publish-snapshot.yml' - '.github/workflows/recurring-jmh-benchmarks.yml' @@ -45,7 +45,7 @@ on: - '.github/workflows/spark-ci.yml' - '.github/workflows/stale.yml' - '.gitignore' - - '.asf.yml' + - '.asf.yaml' - 'dev/**' - 'mr/**' - 'hive3/**' @@ -53,7 +53,6 @@ on: - 'hive-runtime/**' - 'flink/**' - 'kafka-connect/**' - - 'pig/**' - 'docs/**' - 'site/**' - 'open-api/**' diff --git a/.github/workflows/flink-ci.yml b/.github/workflows/flink-ci.yml index 8ed555847861..6171d24e3c62 100644 --- a/.github/workflows/flink-ci.yml +++ b/.github/workflows/flink-ci.yml @@ -37,7 +37,7 @@ on: - '.github/workflows/jmh-benchmarks-ci.yml' - '.github/workflows/kafka-connect-ci.yml' - '.github/workflows/labeler.yml' - - '.github/workflows/licence-check.yml' + - '.github/workflows/license-check.yml' - '.github/workflows/open-api.yml' - '.github/workflows/publish-snapshot.yml' - '.github/workflows/recurring-jmh-benchmarks.yml' @@ -45,7 +45,7 @@ on: - '.github/workflows/spark-ci.yml' - '.github/workflows/stale.yml' - '.gitignore' - - '.asf.yml' + - '.asf.yaml' - 'dev/**' - 'mr/**' - 'hive3/**' @@ -53,7 +53,6 @@ on: - 'hive-runtime/**' - 'kafka-connect/**' - 'spark/**' - - 'pig/**' - 'docs/**' - 'site/**' - 'open-api/**' diff --git a/.github/workflows/hive-ci.yml b/.github/workflows/hive-ci.yml index bcaf62cc07f8..0a9357fddae6 100644 --- a/.github/workflows/hive-ci.yml +++ b/.github/workflows/hive-ci.yml @@ -37,7 +37,7 @@ on: - '.github/workflows/jmh-benchmarks-ci.yml' - '.github/workflows/kafka-connect-ci.yml' - '.github/workflows/labeler.yml' - - '.github/workflows/licence-check.yml' + - '.github/workflows/license-check.yml' - '.github/workflows/open-api.yml' - '.github/workflows/publish-snapshot.yml' - '.github/workflows/recurring-jmh-benchmarks.yml' @@ -45,13 +45,12 @@ on: - '.github/workflows/spark-ci.yml' - '.github/workflows/stale.yml' - '.gitignore' - - '.asf.yml' + - '.asf.yaml' - 'dev/**' - 'arrow/**' - 'spark/**' - 'flink/**' - 'kafka-connect/**' - - 'pig/**' - 'docs/**' - 'site/**' - 'open-api/**' diff --git a/.github/workflows/java-ci.yml b/.github/workflows/java-ci.yml index 1da7a673a865..87f6fcea2e11 100644 --- a/.github/workflows/java-ci.yml +++ b/.github/workflows/java-ci.yml @@ -37,7 +37,7 @@ on: - '.github/workflows/jmh-benchmarks-ci.yml' - '.github/workflows/kafka-connect-ci.yml' - '.github/workflows/labeler.yml' - - '.github/workflows/licence-check.yml' + - '.github/workflows/license-check.yml' - '.github/workflows/open-api.yml' - '.github/workflows/publish-snapshot.yml' - '.github/workflows/recurring-jmh-benchmarks.yml' @@ -45,7 +45,7 @@ on: - '.github/workflows/spark-ci.yml' - '.github/workflows/stale.yml' - '.gitignore' - - '.asf.yml' + - '.asf.yaml' - 'dev/**' - 'docs/**' - 'site/**' diff --git a/.github/workflows/kafka-connect-ci.yml b/.github/workflows/kafka-connect-ci.yml index 98ec18a77953..95c8ee5bfb39 100644 --- a/.github/workflows/kafka-connect-ci.yml +++ b/.github/workflows/kafka-connect-ci.yml @@ -37,7 +37,7 @@ on: - '.github/workflows/java-ci.yml' - '.github/workflows/jmh-benchmarks-ci.yml' - '.github/workflows/labeler.yml' - - '.github/workflows/licence-check.yml' + - '.github/workflows/license-check.yml' - '.github/workflows/open-api.yml' - '.github/workflows/publish-snapshot.yml' - '.github/workflows/recurring-jmh-benchmarks.yml' @@ -45,7 +45,7 @@ on: - '.github/workflows/spark-ci.yml' - '.github/workflows/stale.yml' - '.gitignore' - - '.asf.yml' + - '.asf.yaml' - 'dev/**' - 'mr/**' - 'flink/**' @@ -53,7 +53,6 @@ on: - 'hive3-orc-bundle/**' - 'hive-runtime/**' - 'spark/**' - - 'pig/**' - 'docs/**' - 'site/**' - 'open-api/**' diff --git a/.github/workflows/publish-iceberg-rest-fixture-docker.yml b/.github/workflows/publish-iceberg-rest-fixture-docker.yml new file mode 100644 index 000000000000..4e9aa53e1a27 --- /dev/null +++ b/.github/workflows/publish-iceberg-rest-fixture-docker.yml @@ -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. +# + +name: Build and Push 'iceberg-rest-fixture' Docker Image + +on: + push: + tags: + - 'apache-iceberg-[0-9]+.[0-9]+.[0-9]+' + workflow_dispatch: + +env: + DOCKER_IMAGE_TAG: iceberg-rest-fixture + DOCKER_IMAGE_VERSION: latest + DOCKER_REPOSITORY: apache + +jobs: + build: + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v3 + - uses: actions/setup-java@v4 + with: + distribution: zulu + java-version: 21 + - name: Build Iceberg Open API project + run: ./gradlew :iceberg-open-api:shadowJar + - name: Login to Docker Hub + run: | + docker login -u ${{ secrets.DOCKERHUB_USER }} -p ${{ secrets.DOCKERHUB_TOKEN }} + - name: Set the tagged version + # for tag 'apache-iceberg-1.7.1', publish image 'apache/iceberg-rest-fixture:1.7.1' + if: github.event_name == 'push' && contains(github.ref, 'refs/tags/') + run: | + echo "DOCKER_IMAGE_VERSION=`echo ${{ github.ref }} | tr -d -c 0-9.`" >> "$GITHUB_ENV" + - name: Set up QEMU + uses: docker/setup-qemu-action@v3 + - name: Set up Docker Buildx + uses: docker/setup-buildx-action@v3 + - name: Build and Push + uses: docker/build-push-action@v6 + with: + context: ./ + file: ./docker/iceberg-rest-fixture/Dockerfile + platforms: linux/amd64,linux/arm64 + push: true + tags: ${{ env.DOCKER_REPOSITORY }}/${{ env.DOCKER_IMAGE_TAG }}:${{ env.DOCKER_IMAGE_VERSION }} diff --git a/.github/workflows/spark-ci.yml b/.github/workflows/spark-ci.yml index b5d91d3cc76c..022c61084435 100644 --- a/.github/workflows/spark-ci.yml +++ b/.github/workflows/spark-ci.yml @@ -38,14 +38,14 @@ on: - '.github/workflows/jmh-benchmarks-ci.yml' - '.github/workflows/kafka-connect-ci.yml' - '.github/workflows/labeler.yml' - - '.github/workflows/licence-check.yml' + - '.github/workflows/license-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/stale.yml' - '.gitignore' - - '.asf.yml' + - '.asf.yaml' - 'dev/**' - 'site/**' - 'mr/**' @@ -54,7 +54,6 @@ on: - 'hive-runtime/**' - 'flink/**' - 'kafka-connect/**' - - 'pig/**' - 'docs/**' - 'open-api/**' - 'format/**' diff --git a/.palantir/revapi.yml b/.palantir/revapi.yml index fade79326a49..18c63fbe7bb1 100644 --- a/.palantir/revapi.yml +++ b/.palantir/revapi.yml @@ -1145,6 +1145,32 @@ acceptedBreaks: new: "method org.apache.iceberg.BaseMetastoreOperations.CommitStatus org.apache.iceberg.BaseMetastoreTableOperations::checkCommitStatus(java.lang.String,\ \ org.apache.iceberg.TableMetadata)" justification: "Removing deprecated code" + "1.7.0": + org.apache.iceberg:iceberg-core: + - code: "java.method.removed" + old: "method org.apache.iceberg.deletes.PositionDeleteIndex\ + \ org.apache.iceberg.deletes.Deletes::toPositionIndex(java.lang.CharSequence,\ + \ java.util.List>)" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.deletes.PositionDeleteIndex\ + \ org.apache.iceberg.deletes.Deletes::toPositionIndex(java.lang.CharSequence,\ + \ java.util.List>, java.util.concurrent.ExecutorService)" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.io.CloseableIterable org.apache.iceberg.deletes.Deletes::streamingFilter(org.apache.iceberg.io.CloseableIterable,\ + \ java.util.function.Function, org.apache.iceberg.io.CloseableIterable)" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.io.CloseableIterable org.apache.iceberg.deletes.Deletes::streamingFilter(org.apache.iceberg.io.CloseableIterable,\ + \ java.util.function.Function, org.apache.iceberg.io.CloseableIterable,\ + \ org.apache.iceberg.deletes.DeleteCounter)" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.io.CloseableIterable org.apache.iceberg.deletes.Deletes::streamingMarker(org.apache.iceberg.io.CloseableIterable,\ + \ java.util.function.Function, org.apache.iceberg.io.CloseableIterable,\ + \ java.util.function.Consumer)" + justification: "Removing deprecated code" apache-iceberg-0.14.0: org.apache.iceberg:iceberg-api: - code: "java.class.defaultSerializationChanged" 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/README.md b/README.md index 7d2056077804..394d9aea0d3d 100644 --- a/README.md +++ b/README.md @@ -74,7 +74,6 @@ Iceberg also has modules for adding Iceberg support to processing engines: * `iceberg-spark` is an implementation of Spark's Datasource V2 API for Iceberg with submodules for each spark versions (use runtime jars for a shaded version) * `iceberg-flink` contains classes for integrating with Apache Flink (use iceberg-flink-runtime for a shaded version) * `iceberg-mr` contains an InputFormat and other classes for integrating with Apache Hive -* `iceberg-pig` is an implementation of Pig's LoadFunc API for Iceberg --- **NOTE** @@ -98,3 +97,4 @@ This repository contains the Java implementation of Iceberg. Other implementatio * **Go**: [iceberg-go](https://github.com/apache/iceberg-go) * **PyIceberg** (Python): [iceberg-python](https://github.com/apache/iceberg-python) * **Rust**: [iceberg-rust](https://github.com/apache/iceberg-rust) +* **C++**: [iceberg-cpp](https://github.com/apache/iceberg-cpp) 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/AddedRowsScanTask.java b/api/src/main/java/org/apache/iceberg/AddedRowsScanTask.java index d48b268287c3..506e344d3660 100644 --- a/api/src/main/java/org/apache/iceberg/AddedRowsScanTask.java +++ b/api/src/main/java/org/apache/iceberg/AddedRowsScanTask.java @@ -19,6 +19,7 @@ package org.apache.iceberg; import java.util.List; +import org.apache.iceberg.util.ScanTaskUtil; /** * A scan task for inserts generated by adding a data file to the table. @@ -55,7 +56,7 @@ default ChangelogOperation operation() { @Override default long sizeBytes() { - return length() + deletes().stream().mapToLong(ContentFile::fileSizeInBytes).sum(); + return length() + ScanTaskUtil.contentSizeInBytes(deletes()); } @Override diff --git a/api/src/main/java/org/apache/iceberg/DataFile.java b/api/src/main/java/org/apache/iceberg/DataFile.java index 02ad0aff3128..ea6262afac85 100644 --- a/api/src/main/java/org/apache/iceberg/DataFile.java +++ b/api/src/main/java/org/apache/iceberg/DataFile.java @@ -98,12 +98,27 @@ public interface DataFile extends ContentFile { Types.NestedField SORT_ORDER_ID = optional(140, "sort_order_id", IntegerType.get(), "Sort order ID"); Types.NestedField SPEC_ID = optional(141, "spec_id", IntegerType.get(), "Partition spec ID"); + Types.NestedField REFERENCED_DATA_FILE = + optional( + 143, + "referenced_data_file", + StringType.get(), + "Fully qualified location (URI with FS scheme) of a data file that all deletes reference"); + Types.NestedField CONTENT_OFFSET = + optional( + 144, "content_offset", LongType.get(), "The offset in the file where the content starts"); + Types.NestedField CONTENT_SIZE = + optional( + 145, + "content_size_in_bytes", + LongType.get(), + "The length of referenced content stored in the file"); 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 + // NEXT ID TO ASSIGN: 146 static StructType getType(StructType partitionType) { // IDs start at 100 to leave room for changes to ManifestEntry @@ -124,7 +139,10 @@ static StructType getType(StructType partitionType) { KEY_METADATA, SPLIT_OFFSETS, EQUALITY_IDS, - SORT_ORDER_ID); + SORT_ORDER_ID, + REFERENCED_DATA_FILE, + CONTENT_OFFSET, + CONTENT_SIZE); } /** diff --git a/api/src/main/java/org/apache/iceberg/DeleteFile.java b/api/src/main/java/org/apache/iceberg/DeleteFile.java index 0f8087e6a055..340a00e36b17 100644 --- a/api/src/main/java/org/apache/iceberg/DeleteFile.java +++ b/api/src/main/java/org/apache/iceberg/DeleteFile.java @@ -31,4 +31,37 @@ public interface DeleteFile extends ContentFile { default List splitOffsets() { return null; } + + /** + * Returns the location of a data file that all deletes reference. + * + *

The referenced data file is required for deletion vectors and may be optionally captured for + * position delete files that apply to only one data file. This method always returns null for + * equality delete files. + */ + default String referencedDataFile() { + return null; + } + + /** + * Returns the offset in the file where the content starts. + * + *

The content offset is required for deletion vectors and points to the start of the deletion + * vector blob in the Puffin file, enabling direct access. This method always returns null for + * equality and position delete files. + */ + default Long contentOffset() { + return null; + } + + /** + * Returns the length of referenced content stored in the file. + * + *

The content size is required for deletion vectors and indicates the size of the deletion + * vector blob in the Puffin file, enabling direct access. This method always returns null for + * equality and position delete files. + */ + default Long contentSizeInBytes() { + return null; + } } diff --git a/api/src/main/java/org/apache/iceberg/DeleteFiles.java b/api/src/main/java/org/apache/iceberg/DeleteFiles.java index 8a396920e03b..682ebdd49f0c 100644 --- a/api/src/main/java/org/apache/iceberg/DeleteFiles.java +++ b/api/src/main/java/org/apache/iceberg/DeleteFiles.java @@ -51,7 +51,7 @@ public interface DeleteFiles extends SnapshotUpdate { * @return this for method chaining */ default DeleteFiles deleteFile(DataFile file) { - deleteFile(file.path()); + deleteFile(file.location()); return this; } diff --git a/api/src/main/java/org/apache/iceberg/DeletedDataFileScanTask.java b/api/src/main/java/org/apache/iceberg/DeletedDataFileScanTask.java index 9edd6afd0cea..4b9c1704b9d2 100644 --- a/api/src/main/java/org/apache/iceberg/DeletedDataFileScanTask.java +++ b/api/src/main/java/org/apache/iceberg/DeletedDataFileScanTask.java @@ -19,6 +19,7 @@ package org.apache.iceberg; import java.util.List; +import org.apache.iceberg.util.ScanTaskUtil; /** * A scan task for deletes generated by removing a data file from the table. @@ -54,7 +55,7 @@ default ChangelogOperation operation() { @Override default long sizeBytes() { - return length() + existingDeletes().stream().mapToLong(ContentFile::fileSizeInBytes).sum(); + return length() + ScanTaskUtil.contentSizeInBytes(existingDeletes()); } @Override diff --git a/api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java b/api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java index 131edfddd349..1e0a52a53241 100644 --- a/api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java +++ b/api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java @@ -19,6 +19,7 @@ package org.apache.iceberg; import java.util.List; +import org.apache.iceberg.util.ScanTaskUtil; /** * A scan task for deletes generated by adding delete files to the table. @@ -63,9 +64,9 @@ default ChangelogOperation operation() { @Override default long sizeBytes() { - return length() - + addedDeletes().stream().mapToLong(ContentFile::fileSizeInBytes).sum() - + existingDeletes().stream().mapToLong(ContentFile::fileSizeInBytes).sum(); + long addedDeletesSize = ScanTaskUtil.contentSizeInBytes(addedDeletes()); + long existingDeletesSize = ScanTaskUtil.contentSizeInBytes(existingDeletes()); + return length() + addedDeletesSize + existingDeletesSize; } @Override diff --git a/api/src/main/java/org/apache/iceberg/FileFormat.java b/api/src/main/java/org/apache/iceberg/FileFormat.java index d662437d5ddb..6b41aec42c3e 100644 --- a/api/src/main/java/org/apache/iceberg/FileFormat.java +++ b/api/src/main/java/org/apache/iceberg/FileFormat.java @@ -24,6 +24,7 @@ /** Enum of supported file formats. */ public enum FileFormat { + PUFFIN("puffin", false), ORC("orc", true), PARQUET("parquet", true), AVRO("avro", true), diff --git a/api/src/main/java/org/apache/iceberg/FileScanTask.java b/api/src/main/java/org/apache/iceberg/FileScanTask.java index 5fb4b55459e3..94f153e56052 100644 --- a/api/src/main/java/org/apache/iceberg/FileScanTask.java +++ b/api/src/main/java/org/apache/iceberg/FileScanTask.java @@ -19,6 +19,7 @@ package org.apache.iceberg; import java.util.List; +import org.apache.iceberg.util.ScanTaskUtil; /** A scan task over a range of bytes in a single data file. */ public interface FileScanTask extends ContentScanTask, SplittableScanTask { @@ -36,7 +37,7 @@ default Schema schema() { @Override default long sizeBytes() { - return length() + deletes().stream().mapToLong(ContentFile::fileSizeInBytes).sum(); + return length() + ScanTaskUtil.contentSizeInBytes(deletes()); } @Override diff --git a/api/src/main/java/org/apache/iceberg/ManifestFile.java b/api/src/main/java/org/apache/iceberg/ManifestFile.java index 8f20697c7780..60372636e14e 100644 --- a/api/src/main/java/org/apache/iceberg/ManifestFile.java +++ b/api/src/main/java/org/apache/iceberg/ManifestFile.java @@ -49,7 +49,7 @@ public interface ManifestFile { Types.LongType.get(), "Lowest sequence number in the manifest"); Types.NestedField SNAPSHOT_ID = - optional( + required( 503, "added_snapshot_id", Types.LongType.get(), "Snapshot ID that added the manifest"); Types.NestedField ADDED_FILES_COUNT = optional(504, "added_files_count", Types.IntegerType.get(), "Added entry count"); diff --git a/api/src/main/java/org/apache/iceberg/Schema.java b/api/src/main/java/org/apache/iceberg/Schema.java index 9bcf691f5a03..bd07e9798e9b 100644 --- a/api/src/main/java/org/apache/iceberg/Schema.java +++ b/api/src/main/java/org/apache/iceberg/Schema.java @@ -28,6 +28,7 @@ import java.util.Map; import java.util.Set; import java.util.stream.Collectors; +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.BiMap; @@ -54,8 +55,12 @@ 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); + + @VisibleForTesting static final int DEFAULT_VALUES_MIN_FORMAT_VERSION = 3; + + @VisibleForTesting + static final Map MIN_FORMAT_VERSIONS = + ImmutableMap.of(Type.TypeID.TIMESTAMP_NANO, 3, Type.TypeID.VARIANT, 3); private final StructType struct; private final int schemaId; @@ -586,16 +591,37 @@ private List reassignIds(List columns, TypeUtil.GetID * @param formatVersion table format version */ public static void checkCompatibility(Schema schema, int formatVersion) { - // check the type in each field + // accumulate errors as a treemap to keep them in a reasonable order + Map problems = Maps.newTreeMap(); + + // check each field's type and defaults 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); + if (minFormatVersion != null && formatVersion < minFormatVersion) { + problems.put( + field.fieldId(), + String.format( + "Invalid type for %s: %s is not supported until v%s", + schema.findColumnName(field.fieldId()), field.type(), minFormatVersion)); + } + + if (field.initialDefault() != null && formatVersion < DEFAULT_VALUES_MIN_FORMAT_VERSION) { + problems.put( + field.fieldId(), + String.format( + "Invalid initial default for %s: non-null default (%s) is not supported until v%s", + schema.findColumnName(field.fieldId()), + field.initialDefault(), + DEFAULT_VALUES_MIN_FORMAT_VERSION)); + } + } + + // throw if there are any compatibility problems + if (!problems.isEmpty()) { + throw new IllegalStateException( + String.format( + "Invalid schema for v%s:\n- %s", + formatVersion, Joiner.on("\n- ").join(problems.values()))); } } } diff --git a/api/src/main/java/org/apache/iceberg/UpdateSchema.java b/api/src/main/java/org/apache/iceberg/UpdateSchema.java index afe1891cd530..c84c237f8d8f 100644 --- a/api/src/main/java/org/apache/iceberg/UpdateSchema.java +++ b/api/src/main/java/org/apache/iceberg/UpdateSchema.java @@ -369,7 +369,9 @@ default UpdateSchema updateColumn(String name, Type.PrimitiveType newType, Strin * to create a union schema. * *

For fields with same canonical names in both schemas it is required that the widen types is - * supported using {@link UpdateSchema#updateColumn(String, Type.PrimitiveType)} + * supported using {@link UpdateSchema#updateColumn(String, Type.PrimitiveType)}. Differences in + * type are ignored if the new type is narrower than the existing type (e.g. long to int, double + * to float). * *

Only supports turning a previously required field into an optional one if it is marked * optional in the provided new schema using {@link UpdateSchema#makeColumnOptional(String)} 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 85773febae17..61750d83fc79 100644 --- a/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java +++ b/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java @@ -76,4 +76,16 @@ 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/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/EncryptingFileIO.java b/api/src/main/java/org/apache/iceberg/encryption/EncryptingFileIO.java index 0203361844a5..d3de7b1f84a3 100644 --- a/api/src/main/java/org/apache/iceberg/encryption/EncryptingFileIO.java +++ b/api/src/main/java/org/apache/iceberg/encryption/EncryptingFileIO.java @@ -93,10 +93,9 @@ public InputFile newInputFile(DeleteFile file) { private InputFile newInputFile(ContentFile file) { if (file.keyMetadata() != null) { - return newDecryptingInputFile( - file.path().toString(), file.fileSizeInBytes(), file.keyMetadata()); + return newDecryptingInputFile(file.location(), file.fileSizeInBytes(), file.keyMetadata()); } else { - return newInputFile(file.path().toString(), file.fileSizeInBytes()); + return newInputFile(file.location(), file.fileSizeInBytes()); } } @@ -148,7 +147,7 @@ public void close() { } private SimpleEncryptedInputFile wrap(ContentFile file) { - InputFile encryptedInputFile = io.newInputFile(file.path().toString(), file.fileSizeInBytes()); + InputFile encryptedInputFile = io.newInputFile(file.location(), file.fileSizeInBytes()); return new SimpleEncryptedInputFile(encryptedInputFile, toKeyMetadata(file.keyMetadata())); } 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 82d513ced7dd..68b9e9447926 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java +++ b/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java @@ -337,9 +337,9 @@ public Expression predicate(UnboundPredicate pred) { pred.op(), pred.term(), (T) sanitize(pred.literal(), now, today)); case IN: case NOT_IN: - Iterable iter = - () -> pred.literals().stream().map(lit -> sanitize(lit, now, today)).iterator(); - return new UnboundPredicate<>(pred.op(), pred.term(), (Iterable) iter); + Iterable iter = + () -> pred.literals().stream().map(lit -> (T) sanitize(lit, now, today)).iterator(); + return new UnboundPredicate<>(pred.op(), pred.term(), iter); default: throw new UnsupportedOperationException( "Cannot sanitize unsupported predicate type: " + pred.op()); @@ -534,7 +534,8 @@ private static String sanitize(Type type, Object value, long now, int today) { case DECIMAL: case FIXED: case BINARY: - // for boolean, uuid, decimal, fixed, and binary, match the string result + case VARIANT: + // for boolean, uuid, decimal, fixed, variant, and binary, match the string result return sanitizeSimpleString(value.toString()); } throw new UnsupportedOperationException( @@ -562,7 +563,7 @@ private static String sanitize(Literal literal, long now, int today) { } else if (literal instanceof Literals.DoubleLiteral) { return sanitizeNumber(((Literals.DoubleLiteral) literal).value(), "float"); } else { - // for uuid, decimal, fixed, and binary, match the string result + // for uuid, decimal, fixed, variant, and binary, match the string result return sanitizeSimpleString(literal.value().toString()); } } diff --git a/api/src/main/java/org/apache/iceberg/io/CloseableIterable.java b/api/src/main/java/org/apache/iceberg/io/CloseableIterable.java index 06323612a178..34c561bc373d 100644 --- a/api/src/main/java/org/apache/iceberg/io/CloseableIterable.java +++ b/api/src/main/java/org/apache/iceberg/io/CloseableIterable.java @@ -32,6 +32,23 @@ public interface CloseableIterable extends Iterable, Closeable { + /** + * Adapts an Iterable to CloseableIterable using a no-op close if it is not Closeable. + * + * @param iterable an Iterable + * @return a CloseableIterable that closes Iterable if it is Closeable + */ + static CloseableIterable of(Iterable iterable) { + if (iterable instanceof CloseableIterable) { + return (CloseableIterable) iterable; + } else if (iterable instanceof Closeable) { + Closeable asCloseable = (Closeable) iterable; + return combine(iterable, asCloseable); + } else { + return withNoopClose(iterable); + } + } + /** * Returns a closeable iterator over elements of type {@code T}. * diff --git a/api/src/main/java/org/apache/iceberg/io/FileIO.java b/api/src/main/java/org/apache/iceberg/io/FileIO.java index de4bc2e12a81..f5404b9e5a78 100644 --- a/api/src/main/java/org/apache/iceberg/io/FileIO.java +++ b/api/src/main/java/org/apache/iceberg/io/FileIO.java @@ -50,16 +50,16 @@ default InputFile newInputFile(DataFile file) { Preconditions.checkArgument( file.keyMetadata() == null, "Cannot decrypt data file: %s (use EncryptingFileIO)", - file.path()); - return newInputFile(file.path().toString(), file.fileSizeInBytes()); + file.location()); + return newInputFile(file.location(), file.fileSizeInBytes()); } default InputFile newInputFile(DeleteFile file) { Preconditions.checkArgument( file.keyMetadata() == null, "Cannot decrypt delete file: %s (use EncryptingFileIO)", - file.path()); - return newInputFile(file.path().toString(), file.fileSizeInBytes()); + file.location()); + return newInputFile(file.location(), file.fileSizeInBytes()); } default InputFile newInputFile(ManifestFile manifest) { 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 0e4e782cc110..2b2439e3ed0a 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Bucket.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Bucket.java @@ -43,6 +43,12 @@ static Bucket get(int numBuckets) { return new Bucket<>(numBuckets); } + /** + * Instantiates a new Bucket Transform + * + * @deprecated will be removed in 2.0.0; use {@link #get(int)} instead + */ + @Deprecated @SuppressWarnings("unchecked") static & SerializableFunction> B get( Type type, int numBuckets) { @@ -94,6 +100,14 @@ protected int hash(T value) { "hash(value) is not supported on the base Bucket class"); } + /** + * Transforms a value to its corresponding partition value. + * + * @param value a source value + * @return a transformed partition value + * @deprecated will be removed in 2.0.0; use {@link #bind(Type)} instead + */ + @Deprecated @Override public Integer apply(T value) { if (value == null) { 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 88db16797867..841e6dfa3a51 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Dates.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Dates.java @@ -73,6 +73,14 @@ public Integer apply(Integer days) { this.apply = new Apply(granularity); } + /** + * Transforms a value to its corresponding partition value. + * + * @param days a source value + * @return a transformed partition value + * @deprecated will be removed in 2.0.0; use {@link #bind(Type)} instead + */ + @Deprecated @Override public Integer apply(Integer days) { return apply.apply(days); diff --git a/api/src/main/java/org/apache/iceberg/transforms/Identity.java b/api/src/main/java/org/apache/iceberg/transforms/Identity.java index d4e5e532943b..099a99cc3cf4 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Identity.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Identity.java @@ -38,6 +38,9 @@ class Identity implements Transform { */ @Deprecated public static Identity get(Type type) { + Preconditions.checkArgument( + type.typeId() != Type.TypeID.VARIANT, "Unsupported type for identity: %s", type); + return new Identity<>(type); } @@ -68,6 +71,14 @@ private Identity(Type type) { this.type = type; } + /** + * Transforms a value to its corresponding partition value. + * + * @param value a source value + * @return a transformed partition value + * @deprecated will be removed in 2.0.0; use {@link #bind(Type)} instead + */ + @Deprecated @Override public T apply(T value) { return value; 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 8b8c2ca0a96b..f2c705506305 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Timestamps.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Timestamps.java @@ -52,6 +52,14 @@ enum Timestamps implements Transform { this.apply = apply; } + /** + * Transforms a value to its corresponding partition value. + * + * @param timestamp a source value + * @return a transformed partition value + * @deprecated will be removed in 2.0.0; use {@link #bind(Type)} instead + */ + @Deprecated @Override public Integer apply(Long timestamp) { return apply.apply(timestamp); diff --git a/api/src/main/java/org/apache/iceberg/transforms/UnknownTransform.java b/api/src/main/java/org/apache/iceberg/transforms/UnknownTransform.java index c176fd766a35..aebd3445e36e 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/UnknownTransform.java +++ b/api/src/main/java/org/apache/iceberg/transforms/UnknownTransform.java @@ -33,6 +33,15 @@ public class UnknownTransform implements Transform { this.transform = transform; } + /** + * Transforms a value to its corresponding partition value. + * + * @param value a source value + * @return ∅ + * @throws UnsupportedOperationException Implementation is unknown + * @deprecated will be removed in 2.0.0; use {@link #bind(Type)} instead + */ + @Deprecated @Override public T apply(S value) { throw new UnsupportedOperationException( diff --git a/api/src/main/java/org/apache/iceberg/transforms/VoidTransform.java b/api/src/main/java/org/apache/iceberg/transforms/VoidTransform.java index 5e8e7494c4b5..b46780244faf 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/VoidTransform.java +++ b/api/src/main/java/org/apache/iceberg/transforms/VoidTransform.java @@ -49,6 +49,14 @@ public Void apply(S t) { private VoidTransform() {} + /** + * Transforms a value to its corresponding partition value. + * + * @param value a source value + * @return null + * @deprecated will be removed in 2.0.0; use {@link #bind(Type)} instead + */ + @Deprecated @Override public Void apply(Object value) { return null; @@ -84,6 +92,16 @@ public boolean isVoid() { return true; } + /** + * Returns a human-readable String representation of a transformed value. + * + *

null values will return "null" + * + * @param value a transformed value + * @return a human-readable String representation of null + * @deprecated will be removed in 2.0.0; use {@link #toHumanString(Type, Object)} instead + */ + @Deprecated @Override public String toHumanString(Void value) { return "null"; diff --git a/api/src/main/java/org/apache/iceberg/types/FindTypeVisitor.java b/api/src/main/java/org/apache/iceberg/types/FindTypeVisitor.java index d1dc4adc214a..f0750f337e2e 100644 --- a/api/src/main/java/org/apache/iceberg/types/FindTypeVisitor.java +++ b/api/src/main/java/org/apache/iceberg/types/FindTypeVisitor.java @@ -76,6 +76,15 @@ public Type map(Types.MapType map, Type keyResult, Type valueResult) { return valueResult; } + @Override + public Type variant() { + if (predicate.test(Types.VariantType.get())) { + return Types.VariantType.get(); + } + + return null; + } + @Override public Type primitive(Type.PrimitiveType primitive) { if (predicate.test(primitive)) { diff --git a/api/src/main/java/org/apache/iceberg/types/IndexByName.java b/api/src/main/java/org/apache/iceberg/types/IndexByName.java index 9183ea85f467..131434c9a156 100644 --- a/api/src/main/java/org/apache/iceberg/types/IndexByName.java +++ b/api/src/main/java/org/apache/iceberg/types/IndexByName.java @@ -176,6 +176,11 @@ public Map map( return nameToId; } + @Override + public Map variant() { + return nameToId; + } + @Override public Map primitive(Type.PrimitiveType primitive) { return nameToId; diff --git a/api/src/main/java/org/apache/iceberg/types/IndexParents.java b/api/src/main/java/org/apache/iceberg/types/IndexParents.java index bcd1e1ee900c..952447ed2799 100644 --- a/api/src/main/java/org/apache/iceberg/types/IndexParents.java +++ b/api/src/main/java/org/apache/iceberg/types/IndexParents.java @@ -76,6 +76,11 @@ public Map map( return idToParent; } + @Override + public Map variant() { + return idToParent; + } + @Override public Map primitive(Type.PrimitiveType primitive) { return idToParent; diff --git a/api/src/main/java/org/apache/iceberg/types/ReassignIds.java b/api/src/main/java/org/apache/iceberg/types/ReassignIds.java index d31fea98e53f..565ceee2a901 100644 --- a/api/src/main/java/org/apache/iceberg/types/ReassignIds.java +++ b/api/src/main/java/org/apache/iceberg/types/ReassignIds.java @@ -79,11 +79,7 @@ public Type struct(Types.StructType struct, Iterable fieldTypes) { for (int i = 0; i < length; i += 1) { Types.NestedField field = fields.get(i); int fieldId = id(sourceStruct, field.name()); - if (field.isRequired()) { - newFields.add(Types.NestedField.required(fieldId, field.name(), types.get(i), field.doc())); - } else { - newFields.add(Types.NestedField.optional(fieldId, field.name(), types.get(i), field.doc())); - } + newFields.add(Types.NestedField.from(field).withId(fieldId).ofType(types.get(i)).build()); } return Types.StructType.of(newFields); 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 571bf9a14e43..30870535521f 100644 --- a/api/src/main/java/org/apache/iceberg/types/Type.java +++ b/api/src/main/java/org/apache/iceberg/types/Type.java @@ -45,7 +45,8 @@ enum TypeID { DECIMAL(BigDecimal.class), STRUCT(StructLike.class), LIST(List.class), - MAP(Map.class); + MAP(Map.class), + VARIANT(Object.class); private final Class javaClass; 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 8a9184569aec..7fcf3db3a40d 100644 --- a/api/src/main/java/org/apache/iceberg/types/TypeUtil.java +++ b/api/src/main/java/org/apache/iceberg/types/TypeUtil.java @@ -534,6 +534,7 @@ private static int estimateSize(Type type) { case FIXED: return ((Types.FixedType) type).length(); case BINARY: + case VARIANT: return 80; case DECIMAL: // 12 (header) + (12 + 12 + 4) (BigInteger) + 4 (scale) = 44 bytes @@ -612,6 +613,10 @@ public T map(Types.MapType map, T keyResult, T valueResult) { return null; } + public T variant() { + return null; + } + public T primitive(Type.PrimitiveType primitive) { return null; } @@ -675,6 +680,9 @@ public static T visit(Type type, SchemaVisitor visitor) { return visitor.map(map, keyResult, valueResult); + case VARIANT: + return visitor.variant(); + default: return visitor.primitive(type.asPrimitiveType()); } 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 4bb1674f3be5..3c03a3defb42 100644 --- a/api/src/main/java/org/apache/iceberg/types/Types.java +++ b/api/src/main/java/org/apache/iceberg/types/Types.java @@ -412,6 +412,41 @@ public String toString() { } } + public static class VariantType implements Type { + private static final VariantType INSTANCE = new VariantType(); + + public static VariantType get() { + return INSTANCE; + } + + @Override + public TypeID typeId() { + return TypeID.VARIANT; + } + + @Override + public String toString() { + return "variant"; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } else if (!(o instanceof VariantType)) { + return false; + } + + VariantType that = (VariantType) o; + return typeId() == that.typeId(); + } + + @Override + public int hashCode() { + return Objects.hash(VariantType.class, typeId()); + } + } + public static class DecimalType extends PrimitiveType { public static DecimalType of(int precision, int scale) { return new DecimalType(precision, scale); @@ -676,8 +711,14 @@ public boolean equals(Object o) { return false; } else if (!Objects.equals(doc, that.doc)) { return false; + } else if (!type.equals(that.type)) { + return false; + } else if (!Objects.equals(initialDefault, that.initialDefault)) { + return false; + } else if (!Objects.equals(writeDefault, that.writeDefault)) { + return false; } - return type.equals(that.type); + return true; } @Override diff --git a/api/src/main/java/org/apache/iceberg/util/DeleteFileSet.java b/api/src/main/java/org/apache/iceberg/util/DeleteFileSet.java index bbe9824963fc..06ddd1869ace 100644 --- a/api/src/main/java/org/apache/iceberg/util/DeleteFileSet.java +++ b/api/src/main/java/org/apache/iceberg/util/DeleteFileSet.java @@ -97,13 +97,14 @@ public boolean equals(Object o) { } DeleteFileWrapper that = (DeleteFileWrapper) o; - // this needs to be updated once deletion vector support is added - return Objects.equals(file.location(), that.file.location()); + return Objects.equals(file.location(), that.file.location()) + && Objects.equals(file.contentOffset(), that.file.contentOffset()) + && Objects.equals(file.contentSizeInBytes(), that.file.contentSizeInBytes()); } @Override public int hashCode() { - return Objects.hashCode(file.location()); + return Objects.hash(file.location(), file.contentOffset(), file.contentSizeInBytes()); } @Override diff --git a/api/src/main/java/org/apache/iceberg/util/ScanTaskUtil.java b/api/src/main/java/org/apache/iceberg/util/ScanTaskUtil.java new file mode 100644 index 000000000000..276aae6e2caf --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/util/ScanTaskUtil.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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 org.apache.iceberg.ContentFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileContent; +import org.apache.iceberg.FileFormat; + +public class ScanTaskUtil { + + private ScanTaskUtil() {} + + public static long contentSizeInBytes(ContentFile file) { + if (file.content() == FileContent.DATA) { + return file.fileSizeInBytes(); + } else { + DeleteFile deleteFile = (DeleteFile) file; + return isDV(deleteFile) ? deleteFile.contentSizeInBytes() : deleteFile.fileSizeInBytes(); + } + } + + public static long contentSizeInBytes(Iterable> files) { + long size = 0L; + for (ContentFile file : files) { + size += contentSizeInBytes(file); + } + return size; + } + + private static boolean isDV(DeleteFile deleteFile) { + return deleteFile.format() == FileFormat.PUFFIN; + } +} 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/TestHelpers.java b/api/src/test/java/org/apache/iceberg/TestHelpers.java index ca3b1a908ac6..cf1dc203f4d6 100644 --- a/api/src/test/java/org/apache/iceberg/TestHelpers.java +++ b/api/src/test/java/org/apache/iceberg/TestHelpers.java @@ -53,6 +53,9 @@ public class TestHelpers { private TestHelpers() {} + public static final int MAX_FORMAT_VERSION = 3; + public static final int[] ALL_VERSIONS = IntStream.rangeClosed(1, MAX_FORMAT_VERSION).toArray(); + /** Wait in a tight check loop until system clock is past {@code timestampMillis} */ public static long waitUntilAfter(long timestampMillis) { long current = System.currentTimeMillis(); @@ -646,7 +649,7 @@ public CharSequence path() { @Override public FileFormat format() { - return FileFormat.fromFileName(path()); + return FileFormat.fromFileName(location()); } @Override diff --git a/api/src/test/java/org/apache/iceberg/TestPartitionSpecValidation.java b/api/src/test/java/org/apache/iceberg/TestPartitionSpecValidation.java index 263db427aa16..971f5a9e4510 100644 --- a/api/src/test/java/org/apache/iceberg/TestPartitionSpecValidation.java +++ b/api/src/test/java/org/apache/iceberg/TestPartitionSpecValidation.java @@ -21,6 +21,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.transforms.Transforms; import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.NestedField; @@ -34,7 +35,8 @@ public class TestPartitionSpecValidation { NestedField.required(3, "another_ts", Types.TimestampType.withZone()), NestedField.required(4, "d", Types.TimestampType.withZone()), NestedField.required(5, "another_d", Types.TimestampType.withZone()), - NestedField.required(6, "s", Types.StringType.get())); + NestedField.required(6, "s", Types.StringType.get()), + NestedField.required(7, "v", Types.VariantType.get())); @Test public void testMultipleTimestampPartitions() { @@ -312,4 +314,15 @@ public void testAddPartitionFieldsWithAndWithoutFieldIds() { assertThat(spec.fields().get(2).fieldId()).isEqualTo(1006); assertThat(spec.lastAssignedFieldId()).isEqualTo(1006); } + + @Test + public void testVariantUnsupported() { + assertThatThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA) + .add(7, 1005, "variant_partition1", Transforms.bucket(5)) + .build()) + .isInstanceOf(ValidationException.class) + .hasMessage("Cannot partition by non-primitive source field: variant"); + } } diff --git a/api/src/test/java/org/apache/iceberg/TestSchema.java b/api/src/test/java/org/apache/iceberg/TestSchema.java new file mode 100644 index 000000000000..e9cb387eebb5 --- /dev/null +++ b/api/src/test/java/org/apache/iceberg/TestSchema.java @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import static org.apache.iceberg.Schema.DEFAULT_VALUES_MIN_FORMAT_VERSION; +import static org.apache.iceberg.Schema.MIN_FORMAT_VERSIONS; +import static org.apache.iceberg.TestHelpers.MAX_FORMAT_VERSION; +import static org.assertj.core.api.Assertions.assertThatCode; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.List; +import java.util.stream.IntStream; +import java.util.stream.Stream; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.FieldSource; +import org.junit.jupiter.params.provider.MethodSource; + +public class TestSchema { + + private static final List TEST_TYPES = + ImmutableList.of( + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), + Types.VariantType.get()); + + private static final Schema INITIAL_DEFAULT_SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.required("has_default") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("--") + .withWriteDefault("--") + .build()); + + private static final Schema WRITE_DEFAULT_SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.required("has_default") + .withId(2) + .ofType(Types.StringType.get()) + .withWriteDefault("--") + .build()); + + private Schema generateTypeSchema(Type type) { + return new Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "top", type), + Types.NestedField.optional(3, "arr", Types.ListType.ofRequired(4, type)), + Types.NestedField.required( + 5, + "struct", + Types.StructType.of( + Types.NestedField.optional(6, "inner_op", type), + Types.NestedField.required(7, "inner_req", type), + Types.NestedField.optional( + 8, + "struct_arr", + Types.StructType.of(Types.NestedField.optional(9, "deep", type)))))); + } + + private static Stream unsupportedTypes() { + return TEST_TYPES.stream() + .flatMap( + type -> + IntStream.range(1, MIN_FORMAT_VERSIONS.get(type.typeId())) + .mapToObj(unsupportedVersion -> Arguments.of(type, unsupportedVersion))); + } + + @ParameterizedTest + @MethodSource("unsupportedTypes") + public void testUnsupportedTypes(Type type, int unsupportedVersion) { + assertThatThrownBy( + () -> Schema.checkCompatibility(generateTypeSchema(type), unsupportedVersion)) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Invalid schema for v%s:\n" + + "- Invalid type for top: %s is not supported until v%s\n" + + "- Invalid type for arr.element: %s is not supported until v%s\n" + + "- Invalid type for struct.inner_op: %s is not supported until v%s\n" + + "- Invalid type for struct.inner_req: %s is not supported until v%s\n" + + "- Invalid type for struct.struct_arr.deep: %s is not supported until v%s", + unsupportedVersion, + type, + MIN_FORMAT_VERSIONS.get(type.typeId()), + type, + MIN_FORMAT_VERSIONS.get(type.typeId()), + type, + MIN_FORMAT_VERSIONS.get(type.typeId()), + type, + MIN_FORMAT_VERSIONS.get(type.typeId()), + type, + MIN_FORMAT_VERSIONS.get(type.typeId())); + } + + private static Stream supportedTypes() { + return TEST_TYPES.stream() + .flatMap( + type -> + IntStream.rangeClosed(MIN_FORMAT_VERSIONS.get(type.typeId()), MAX_FORMAT_VERSION) + .mapToObj(supportedVersion -> Arguments.of(type, supportedVersion))); + } + + @ParameterizedTest + @MethodSource("supportedTypes") + public void testTypeSupported(Type type, int supportedVersion) { + assertThatCode(() -> Schema.checkCompatibility(generateTypeSchema(type), supportedVersion)) + .doesNotThrowAnyException(); + } + + private static int[] unsupportedInitialDefault = + IntStream.range(1, DEFAULT_VALUES_MIN_FORMAT_VERSION).toArray(); + + @ParameterizedTest + @FieldSource("unsupportedInitialDefault") + public void testUnsupportedInitialDefault(int formatVersion) { + assertThatThrownBy(() -> Schema.checkCompatibility(INITIAL_DEFAULT_SCHEMA, formatVersion)) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Invalid schema for v%s:\n" + + "- Invalid initial default for has_default: " + + "non-null default (--) is not supported until v3", + formatVersion); + } + + private static int[] supportedInitialDefault = + IntStream.rangeClosed(DEFAULT_VALUES_MIN_FORMAT_VERSION, MAX_FORMAT_VERSION).toArray(); + + @ParameterizedTest + @FieldSource("supportedInitialDefault") + public void testSupportedInitialDefault(int formatVersion) { + assertThatCode(() -> Schema.checkCompatibility(INITIAL_DEFAULT_SCHEMA, formatVersion)) + .doesNotThrowAnyException(); + } + + @ParameterizedTest + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") + public void testSupportedWriteDefault(int formatVersion) { + // only the initial default is a forward-incompatible change + assertThatCode(() -> Schema.checkCompatibility(WRITE_DEFAULT_SCHEMA, formatVersion)) + .doesNotThrowAnyException(); + } +} 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 fc4333d7c6c5..5f0cac2b5e8c 100644 --- a/api/src/test/java/org/apache/iceberg/transforms/TestBucketing.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestBucketing.java @@ -417,6 +417,20 @@ public void testVerifiedIllegalNumBuckets() { .hasMessage("Invalid number of buckets: 0 (must be > 0)"); } + @Test + public void testVariantUnsupported() { + assertThatThrownBy(() -> Transforms.bucket(Types.VariantType.get(), 3)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot bucket by type: variant"); + + Transform bucket = Transforms.bucket(3); + assertThatThrownBy(() -> bucket.bind(Types.VariantType.get())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot bucket by type: variant"); + + assertThat(bucket.canTransform(Types.VariantType.get())).isFalse(); + } + private byte[] randomBytes(int length) { byte[] bytes = new byte[length]; testRandom.nextBytes(bytes); 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 93d3281411f3..b5076e08a947 100644 --- a/api/src/test/java/org/apache/iceberg/transforms/TestIdentity.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestIdentity.java @@ -19,6 +19,7 @@ package org.apache.iceberg.transforms; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.math.BigDecimal; import java.nio.ByteBuffer; @@ -155,4 +156,21 @@ public void testBigDecimalToHumanString() { .as("Should not modify Strings") .isEqualTo(decimalString); } + + @Test + public void testVariantUnsupported() { + assertThatThrownBy(() -> Transforms.identity().bind(Types.VariantType.get())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot bind to unsupported type: variant"); + + assertThatThrownBy(() -> Transforms.fromString(Types.VariantType.get(), "identity")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Unsupported type for identity: variant"); + + assertThatThrownBy(() -> Transforms.identity(Types.VariantType.get())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Unsupported type for identity: variant"); + + assertThat(Transforms.identity().canTransform(Types.VariantType.get())).isFalse(); + } } 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 96c330d6eb43..af2ebae7e1a8 100644 --- a/api/src/test/java/org/apache/iceberg/types/TestSerializableTypes.java +++ b/api/src/test/java/org/apache/iceberg/types/TestSerializableTypes.java @@ -45,7 +45,7 @@ public void testIdentityTypes() throws Exception { Types.TimestampNanoType.withZone(), Types.StringType.get(), Types.UUIDType.get(), - Types.BinaryType.get() + Types.BinaryType.get(), }; for (Type type : identityPrimitives) { @@ -112,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()) @@ -127,6 +127,15 @@ public void testLists() throws Exception { } } + @Test + public void testVariant() throws Exception { + Types.VariantType variant = Types.VariantType.get(); + Type copy = TestHelpers.roundTripSerialize(variant); + assertThat(copy) + .as("Variant serialization should be equal to starting type") + .isEqualTo(variant); + } + @Test public void testSchema() throws Exception { Schema schema = diff --git a/api/src/test/java/org/apache/iceberg/types/TestTypeUtil.java b/api/src/test/java/org/apache/iceberg/types/TestTypeUtil.java index e8db0937eb73..36384d232af3 100644 --- a/api/src/test/java/org/apache/iceberg/types/TestTypeUtil.java +++ b/api/src/test/java/org/apache/iceberg/types/TestTypeUtil.java @@ -595,7 +595,12 @@ public void testReassignOrRefreshIds() { new Schema( Lists.newArrayList( required(10, "a", Types.IntegerType.get()), - required(11, "c", Types.IntegerType.get()), + Types.NestedField.required("c") + .withId(11) + .ofType(Types.IntegerType.get()) + .withInitialDefault(23) + .withWriteDefault(34) + .build(), required(12, "B", Types.IntegerType.get())), Sets.newHashSet(10)); Schema sourceSchema = @@ -603,13 +608,20 @@ public void testReassignOrRefreshIds() { Lists.newArrayList( required(1, "a", Types.IntegerType.get()), required(15, "B", Types.IntegerType.get()))); - final Schema actualSchema = TypeUtil.reassignOrRefreshIds(schema, sourceSchema); - final Schema expectedSchema = + + Schema actualSchema = TypeUtil.reassignOrRefreshIds(schema, sourceSchema); + Schema expectedSchema = new Schema( Lists.newArrayList( required(1, "a", Types.IntegerType.get()), - required(16, "c", Types.IntegerType.get()), + Types.NestedField.required("c") + .withId(16) + .ofType(Types.IntegerType.get()) + .withInitialDefault(23) + .withWriteDefault(34) + .build(), required(15, "B", Types.IntegerType.get()))); + assertThat(actualSchema.asStruct()).isEqualTo(expectedSchema.asStruct()); } diff --git a/api/src/test/java/org/apache/iceberg/util/RandomUtil.java b/api/src/test/java/org/apache/iceberg/util/RandomUtil.java index 9131e6166133..42dd6825a4d5 100644 --- a/api/src/test/java/org/apache/iceberg/util/RandomUtil.java +++ b/api/src/test/java/org/apache/iceberg/util/RandomUtil.java @@ -200,7 +200,10 @@ public static Object generateDictionaryEncodablePrimitive( "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789_-.!?"; private static String randomString(Random random) { - int length = random.nextInt(50); + return generateString(random.nextInt(50), random); + } + + public static String generateString(int length, Random random) { byte[] buffer = new byte[length]; for (int i = 0; i < length; i += 1) { diff --git a/api/src/test/java/org/apache/iceberg/util/TestExceptionUtil.java b/api/src/test/java/org/apache/iceberg/util/TestExceptionUtil.java index e6c3cf5c20ba..20315176b11a 100644 --- a/api/src/test/java/org/apache/iceberg/util/TestExceptionUtil.java +++ b/api/src/test/java/org/apache/iceberg/util/TestExceptionUtil.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.util.Arrays; +import org.assertj.core.api.InstanceOfAssertFactories; import org.junit.jupiter.api.Test; public class TestExceptionUtil { @@ -53,7 +54,7 @@ public void testRunSafely() { .isInstanceOf(CustomCheckedException.class) .isEqualTo(exc) .extracting(e -> Arrays.asList(e.getSuppressed())) - .asList() + .asInstanceOf(InstanceOfAssertFactories.LIST) .hasSize(2) .containsExactly(suppressedOne, suppressedTwo); } @@ -82,7 +83,7 @@ public void testRunSafelyTwoExceptions() { .isInstanceOf(CustomCheckedException.class) .isEqualTo(exc) .extracting(e -> Arrays.asList(e.getSuppressed())) - .asList() + .asInstanceOf(InstanceOfAssertFactories.LIST) .hasSize(2) .containsExactly(suppressedOne, suppressedTwo); } @@ -112,7 +113,7 @@ public void testRunSafelyThreeExceptions() { .isInstanceOf(CustomCheckedException.class) .isEqualTo(exc) .extracting(e -> Arrays.asList(e.getSuppressed())) - .asList() + .asInstanceOf(InstanceOfAssertFactories.LIST) .hasSize(2) .containsExactly(suppressedOne, suppressedTwo); } @@ -137,7 +138,7 @@ public void testRunSafelyRuntimeExceptions() { .isInstanceOf(RuntimeException.class) .isEqualTo(exc) .extracting(e -> Arrays.asList(e.getSuppressed())) - .asList() + .asInstanceOf(InstanceOfAssertFactories.LIST) .hasSize(2) .containsExactly(suppressedOne, suppressedTwo); } diff --git a/api/src/test/java/org/apache/iceberg/util/TestScanTaskUtil.java b/api/src/test/java/org/apache/iceberg/util/TestScanTaskUtil.java new file mode 100644 index 000000000000..a449cf20a65b --- /dev/null +++ b/api/src/test/java/org/apache/iceberg/util/TestScanTaskUtil.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.util; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; + +public class TestScanTaskUtil { + + @Test + public void testContentSize() { + DeleteFile dv1 = mockDV("dv1.puffin", 20L, 25L, "data1.parquet"); + DeleteFile dv2 = mockDV("dv2.puffin", 4L, 15L, "data2.parquet"); + + long size1 = ScanTaskUtil.contentSizeInBytes(ImmutableList.of()); + assertThat(size1).isEqualTo(0); + + long size2 = ScanTaskUtil.contentSizeInBytes(ImmutableList.of(dv1)); + assertThat(size2).isEqualTo(25L); + + long size3 = ScanTaskUtil.contentSizeInBytes(ImmutableList.of(dv1, dv2)); + assertThat(size3).isEqualTo(40L); + } + + private static DeleteFile mockDV( + String location, long contentOffset, long contentSize, String referencedDataFile) { + DeleteFile mockFile = Mockito.mock(DeleteFile.class); + Mockito.when(mockFile.format()).thenReturn(FileFormat.PUFFIN); + Mockito.when(mockFile.location()).thenReturn(location); + Mockito.when(mockFile.contentOffset()).thenReturn(contentOffset); + Mockito.when(mockFile.contentSizeInBytes()).thenReturn(contentSize); + Mockito.when(mockFile.referencedDataFile()).thenReturn(referencedDataFile); + return mockFile; + } +} diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java index dd2b1e0221e4..7bd744ec5bf3 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java @@ -260,7 +260,7 @@ private static final class VectorizedCombinedScanIterator Map keyMetadata = Maps.newHashMap(); fileTasks.stream() .map(FileScanTask::file) - .forEach(file -> keyMetadata.put(file.path().toString(), file.keyMetadata())); + .forEach(file -> keyMetadata.put(file.location(), file.keyMetadata())); Stream encrypted = keyMetadata.entrySet().stream() @@ -364,7 +364,7 @@ public void close() throws IOException { private InputFile getInputFile(FileScanTask task) { Preconditions.checkArgument(!task.isDataTask(), "Invalid task type"); - return inputFiles.get(task.file().path().toString()); + return inputFiles.get(task.file().location()); } /** diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/BaseBatchReader.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/BaseBatchReader.java index 2175293ab2b6..daa116f292ed 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/BaseBatchReader.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/BaseBatchReader.java @@ -42,9 +42,15 @@ protected BaseBatchReader(List> readers) { @Override public void setRowGroupInfo( PageReadStore pageStore, Map metaData, long rowPosition) { + setRowGroupInfo(pageStore, metaData); + } + + @Override + public void setRowGroupInfo( + PageReadStore pageStore, Map metaData) { for (VectorizedArrowReader reader : readers) { if (reader != null) { - reader.setRowGroupInfo(pageStore, metaData, rowPosition); + reader.setRowGroupInfo(pageStore, metaData); } } } diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java index 27ee25124f16..411f241e169f 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java @@ -432,6 +432,11 @@ private void allocateVectorBasedOnTypeName(PrimitiveType primitive, Field arrowF @Override public void setRowGroupInfo( PageReadStore source, Map metadata, long rowPosition) { + setRowGroupInfo(source, metadata); + } + + @Override + public void setRowGroupInfo(PageReadStore source, Map metadata) { ColumnChunkMetaData chunkMetaData = metadata.get(ColumnPath.get(columnDescriptor.getPath())); this.dictionary = vectorizedColumnIterator.setRowGroupInfo( @@ -475,6 +480,10 @@ public VectorHolder read(VectorHolder reuse, int numValsToRead) { public void setRowGroupInfo( PageReadStore source, Map metadata, long rowPosition) {} + @Override + public void setRowGroupInfo( + PageReadStore source, Map metadata) {} + @Override public String toString() { return "NullReader"; @@ -541,7 +550,19 @@ private static NullabilityHolder newNullabilityHolder(int size) { @Override public void setRowGroupInfo( PageReadStore source, Map metadata, long rowPosition) { - this.rowStart = rowPosition; + setRowGroupInfo(source, metadata); + } + + @Override + public void setRowGroupInfo( + PageReadStore source, Map metadata) { + this.rowStart = + source + .getRowIndexOffset() + .orElseThrow( + () -> + new IllegalArgumentException( + "PageReadStore does not contain row index offset")); } @Override @@ -586,6 +607,10 @@ public VectorHolder read(VectorHolder reuse, int numValsToRead) { public void setRowGroupInfo( PageReadStore source, Map metadata, long rowPosition) {} + @Override + public void setRowGroupInfo( + PageReadStore source, Map metadata) {} + @Override public String toString() { return String.format("ConstantReader: %s", value); @@ -613,6 +638,10 @@ public VectorHolder read(VectorHolder reuse, int numValsToRead) { public void setRowGroupInfo( PageReadStore source, Map metadata, long rowPosition) {} + @Override + public void setRowGroupInfo( + PageReadStore source, Map metadata) {} + @Override public String toString() { return "DeletedVectorReader"; diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedReaderBuilder.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedReaderBuilder.java index 3915ff1f1a32..398f42eb1ce7 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedReaderBuilder.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedReaderBuilder.java @@ -20,6 +20,7 @@ import java.util.List; import java.util.Map; +import java.util.function.BiFunction; import java.util.function.Function; import java.util.stream.IntStream; import org.apache.arrow.memory.BufferAllocator; @@ -47,6 +48,7 @@ public class VectorizedReaderBuilder extends TypeWithSchemaVisitor idToConstant; private final boolean setArrowValidityVector; private final Function>, VectorizedReader> readerFactory; + private final BiFunction convert; public VectorizedReaderBuilder( Schema expectedSchema, @@ -54,6 +56,22 @@ public VectorizedReaderBuilder( boolean setArrowValidityVector, Map idToConstant, Function>, VectorizedReader> readerFactory) { + this( + expectedSchema, + parquetSchema, + setArrowValidityVector, + idToConstant, + readerFactory, + (type, value) -> value); + } + + protected VectorizedReaderBuilder( + Schema expectedSchema, + MessageType parquetSchema, + boolean setArrowValidityVector, + Map idToConstant, + Function>, VectorizedReader> readerFactory, + BiFunction convert) { this.parquetSchema = parquetSchema; this.icebergSchema = expectedSchema; this.rootAllocator = @@ -62,6 +80,7 @@ public VectorizedReaderBuilder( this.setArrowValidityVector = setArrowValidityVector; this.idToConstant = idToConstant; this.readerFactory = readerFactory; + this.convert = convert; } @Override @@ -85,7 +104,7 @@ public VectorizedReader message( int id = field.fieldId(); VectorizedReader reader = readersById.get(id); if (idToConstant.containsKey(id)) { - reorderedFields.add(new ConstantVectorReader<>(field, idToConstant.get(id))); + reorderedFields.add(constantReader(field, idToConstant.get(id))); } else if (id == MetadataColumns.ROW_POSITION.fieldId()) { if (setArrowValidityVector) { reorderedFields.add(VectorizedArrowReader.positionsWithSetArrowValidityVector()); @@ -96,13 +115,23 @@ public VectorizedReader message( reorderedFields.add(new DeletedVectorReader()); } else if (reader != null) { reorderedFields.add(reader); - } else { + } else if (field.initialDefault() != null) { + reorderedFields.add( + constantReader(field, convert.apply(field.type(), field.initialDefault()))); + } else if (field.isOptional()) { reorderedFields.add(VectorizedArrowReader.nulls()); + } else { + throw new IllegalArgumentException( + String.format("Missing required field: %s", field.name())); } } return vectorizedReader(reorderedFields); } + private ConstantVectorReader constantReader(Types.NestedField field, T constant) { + return new ConstantVectorReader<>(field, constant); + } + protected VectorizedReader vectorizedReader(List> reorderedFields) { return readerFactory.apply(reorderedFields); } 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 54445e424508..8e52fd1a28c3 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,24 +214,6 @@ 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( - final FieldVector vector, - final int expectedBatchSize, - final int numValsInVector, - final int typeWidth, - NullabilityHolder holder) { - return vectorizedPageIterator - .fixedWidthBinaryPageReader() - .nextBatch(vector, expectedBatchSize, numValsInVector, typeWidth, holder); - } - } - public class BooleanBatchReader extends BatchReader { @Override protected int nextBatchOf( @@ -282,14 +264,6 @@ 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(); - } - public BooleanBatchReader booleanBatchReader() { return new BooleanBatchReader(); } 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 4499d0536867..1a00137fad70 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 @@ -58,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) { @@ -94,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)); } } @@ -102,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); } } @@ -113,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); } } @@ -121,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)); } } @@ -129,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)); } } @@ -137,20 +133,7 @@ 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)); - } - } - - /** - * @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); + vector.getDataBuffer().setDouble((long) idx * typeWidth, dict.decodeToDouble(currentVal)); } } @@ -207,14 +190,6 @@ 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 VarWidthBinaryDictEncodedReader varWidthBinaryDictEncodedReader() { return new VarWidthBinaryDictEncodedReader(); } 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 d36521273d63..b97eb1545550 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 @@ -21,7 +21,6 @@ import java.io.IOException; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.IntVector; -import org.apache.arrow.vector.VarBinaryVector; import org.apache.iceberg.arrow.vectorized.NullabilityHolder; import org.apache.iceberg.parquet.BasePageIterator; import org.apache.iceberg.parquet.ParquetUtil; @@ -417,39 +416,6 @@ 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( - FieldVector vector, int batchSize, int numVals, int typeWidth, NullabilityHolder holder) { - vectorizedDefinitionLevelReader - .fixedWidthBinaryReader() - .nextBatch(vector, numVals, typeWidth, batchSize, holder, plainValuesReader); - } - - @Override - protected void nextDictEncodedVal( - FieldVector vector, int batchSize, int numVals, int typeWidth, NullabilityHolder holder) { - vectorizedDefinitionLevelReader - .fixedWidthBinaryReader() - .nextDictEncodedBatch( - vector, - numVals, - typeWidth, - batchSize, - holder, - dictionaryEncodedValuesReader, - dictionary); - } - } - /** Method for reading batches of booleans. */ class BooleanPageReader extends BasePageReader { @Override @@ -499,14 +465,6 @@ VarWidthTypePageReader varWidthTypePageReader() { return new VarWidthTypePageReader(); } - /** - * @deprecated since 1.7.0, will be removed in 1.8.0. - */ - @Deprecated - FixedWidthBinaryPageReader fixedWidthBinaryPageReader() { - return new FixedWidthBinaryPageReader(); - } - BooleanPageReader booleanPageReader() { return new BooleanPageReader(); } 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 e8ec7bee8f62..4df648ee3dfd 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 @@ -27,7 +27,6 @@ import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.FixedSizeBinaryVector; import org.apache.arrow.vector.IntVector; -import org.apache.arrow.vector.VarBinaryVector; import org.apache.iceberg.arrow.vectorized.NullabilityHolder; import org.apache.iceberg.parquet.ParquetUtil; import org.apache.iceberg.parquet.ValuesAsBytesReader; @@ -493,48 +492,6 @@ protected void nextDictEncodedVal( } } - /** - * @deprecated since 1.7.0, will be removed in 1.8.0. - */ - @Deprecated - class FixedWidthBinaryReader extends BaseReader { - @Override - protected void nextVal( - FieldVector vector, - int idx, - ValuesAsBytesReader valuesReader, - int typeWidth, - byte[] byteArray) { - ByteBuffer buffer = valuesReader.getBuffer(typeWidth); - ((VarBinaryVector) vector) - .setSafe( - idx, - buffer.array(), - buffer.position() + buffer.arrayOffset(), - buffer.limit() - buffer.position()); - } - - @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 - .fixedWidthBinaryDictEncodedReader() - .nextBatch(vector, idx, numValuesToRead, dict, nullabilityHolder, typeWidth); - } else if (Mode.PACKED.equals(mode)) { - ByteBuffer buffer = dict.decodeToBinary(reader.readInteger()).toByteBuffer(); - vector.getDataBuffer().setBytes((long) idx * typeWidth, buffer); - } - } - } - class FixedSizeBinaryReader extends BaseReader { @Override protected void nextVal( @@ -736,14 +693,6 @@ TimestampInt96Reader timestampInt96Reader() { return new TimestampInt96Reader(); } - /** - * @deprecated since 1.7.0, will be removed in 1.8.0. - */ - @Deprecated - FixedWidthBinaryReader fixedWidthBinaryReader() { - return new FixedWidthBinaryReader(); - } - FixedSizeBinaryReader fixedSizeBinaryReader() { return new FixedSizeBinaryReader(); } 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 e9cf474addfa..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() {} @@ -106,17 +109,16 @@ public static String testMultiRegionAccessPointAlias() { return System.getenv("AWS_TEST_MULTI_REGION_ACCESS_POINT_ALIAS"); } - public static void cleanS3Bucket(S3Client s3, String bucketName, String prefix) { + 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(); } @@ -127,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/glue/GlueTestBase.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java index 29076369c8f5..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 @@ -110,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() { 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 41a07401a1e6..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 @@ -108,25 +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); - multiRegionAccessPointAlias = AwsIntegTestUtil.testMultiRegionAccessPointAlias(); - 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 @@ -171,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)); @@ -201,12 +209,14 @@ public void testCrossRegionAccessEnabled() throws Exception { S3FileIO s3FileIO = new S3FileIO(clientFactory::s3); validateRead(s3FileIO, crossBucketObjectUri); } finally { - AwsIntegTestUtil.cleanS3Bucket(s3Client, crossRegionBucketName, crossBucketObjectKey); + 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( @@ -258,6 +268,7 @@ public void testNewOutputStream() throws Exception { @Test public void testNewOutputStreamWithAccessPoint() throws Exception { + requireAccessPointSupport(); S3FileIO s3FileIO = new S3FileIO(clientFactory::s3); s3FileIO.initialize( ImmutableMap.of( @@ -273,6 +284,7 @@ public void testNewOutputStreamWithAccessPoint() throws Exception { @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); @@ -327,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); @@ -342,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); @@ -363,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); @@ -378,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()); @@ -413,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); @@ -444,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( @@ -454,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( @@ -515,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); @@ -527,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() @@ -613,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 4c3c305d4bae..4149d795d30d 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactory.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactory.java @@ -38,6 +38,7 @@ public class AssumeRoleAwsClientFactory implements AwsClientFactory { private HttpClientProperties httpClientProperties; private S3FileIOProperties s3FileIOProperties; private String roleSessionName; + private AwsClientProperties awsClientProperties; @Override public S3Client s3() { @@ -64,6 +65,7 @@ public KmsClient kms() { return KmsClient.builder() .applyMutation(this::applyAssumeRoleConfigurations) .applyMutation(httpClientProperties::applyHttpClientConfigurations) + .applyMutation(awsClientProperties::applyRetryConfigurations) .build(); } @@ -81,6 +83,7 @@ public void initialize(Map properties) { this.awsProperties = new AwsProperties(properties); this.s3FileIOProperties = new S3FileIOProperties(properties); this.httpClientProperties = new HttpClientProperties(properties); + this.awsClientProperties = new AwsClientProperties(properties); this.roleSessionName = genSessionName(); Preconditions.checkNotNull( awsProperties.clientAssumeRoleArn(), @@ -126,6 +129,10 @@ protected S3FileIOProperties s3FileIOProperties() { return s3FileIOProperties; } + protected AwsClientProperties awsClientProperties() { + return awsClientProperties; + } + private StsClient sts() { return StsClient.builder() .applyMutation(httpClientProperties::applyHttpClientConfigurations) 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 5974e21209e3..7554b5629be4 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java @@ -134,6 +134,7 @@ public KmsClient kms() { .applyMutation(awsClientProperties::applyClientRegionConfiguration) .applyMutation(httpClientProperties::applyHttpClientConfigurations) .applyMutation(awsClientProperties::applyClientCredentialConfigurations) + .applyMutation(awsClientProperties::applyRetryConfigurations) .build(); } diff --git a/aws/src/main/java/org/apache/iceberg/aws/AwsClientProperties.java b/aws/src/main/java/org/apache/iceberg/aws/AwsClientProperties.java index 0c91f8685ae9..abdea1ae9fcd 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AwsClientProperties.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AwsClientProperties.java @@ -20,6 +20,7 @@ import java.io.Serializable; import java.util.Map; +import org.apache.iceberg.aws.s3.VendedCredentialsProvider; import org.apache.iceberg.common.DynClasses; import org.apache.iceberg.common.DynMethods; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -31,6 +32,8 @@ import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; import software.amazon.awssdk.awscore.client.builder.AwsClientBuilder; +import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; +import software.amazon.awssdk.core.retry.RetryMode; import software.amazon.awssdk.regions.Region; public class AwsClientProperties implements Serializable { @@ -66,14 +69,27 @@ public class AwsClientProperties implements Serializable { */ public static final String CLIENT_REGION = "client.region"; + /** + * When set, the {@link VendedCredentialsProvider} will be used to fetch and refresh vended + * credentials from this endpoint. + */ + public static final String REFRESH_CREDENTIALS_ENDPOINT = "client.refresh-credentials-endpoint"; + + /** Controls whether vended credentials should be refreshed or not. Defaults to true. */ + public static final String REFRESH_CREDENTIALS_ENABLED = "client.refresh-credentials-enabled"; + private String clientRegion; private final String clientCredentialsProvider; private final Map clientCredentialsProviderProperties; + private final String refreshCredentialsEndpoint; + private final boolean refreshCredentialsEnabled; public AwsClientProperties() { this.clientRegion = null; this.clientCredentialsProvider = null; this.clientCredentialsProviderProperties = null; + this.refreshCredentialsEndpoint = null; + this.refreshCredentialsEnabled = true; } public AwsClientProperties(Map properties) { @@ -81,6 +97,9 @@ public AwsClientProperties(Map properties) { this.clientCredentialsProvider = properties.get(CLIENT_CREDENTIALS_PROVIDER); this.clientCredentialsProviderProperties = PropertyUtil.propertiesWithPrefix(properties, CLIENT_CREDENTIAL_PROVIDER_PREFIX); + this.refreshCredentialsEndpoint = properties.get(REFRESH_CREDENTIALS_ENDPOINT); + this.refreshCredentialsEnabled = + PropertyUtil.propertyAsBoolean(properties, REFRESH_CREDENTIALS_ENABLED, true); } public String clientRegion() { @@ -122,11 +141,12 @@ public void applyClientCredentialConfigurations(T b } /** - * Returns a credentials provider instance. If params were set, we return a new credentials - * instance. If none of the params are set, we try to dynamically load the provided credentials - * provider class. Upon loading the class, we try to invoke {@code create(Map)} - * static method. If that fails, we fall back to {@code create()}. If credential provider class - * wasn't set, we fall back to default credentials provider. + * Returns a credentials provider instance. If {@link #refreshCredentialsEndpoint} is set, an + * instance of {@link VendedCredentialsProvider} is returned. If params were set, we return a new + * credentials instance. If none of the params are set, we try to dynamically load the provided + * credentials provider class. Upon loading the class, we try to invoke {@code create(Map)} static method. If that fails, we fall back to {@code create()}. If credential + * provider class wasn't set, we fall back to default credentials provider. * * @param accessKeyId the AWS access key ID * @param secretAccessKey the AWS secret access key @@ -136,6 +156,12 @@ public void applyClientCredentialConfigurations(T b @SuppressWarnings("checkstyle:HiddenField") public AwsCredentialsProvider credentialsProvider( String accessKeyId, String secretAccessKey, String sessionToken) { + if (refreshCredentialsEnabled && !Strings.isNullOrEmpty(refreshCredentialsEndpoint)) { + clientCredentialsProviderProperties.put( + VendedCredentialsProvider.URI, refreshCredentialsEndpoint); + return credentialsProvider(VendedCredentialsProvider.class.getName()); + } + if (!Strings.isNullOrEmpty(accessKeyId) && !Strings.isNullOrEmpty(secretAccessKey)) { if (Strings.isNullOrEmpty(sessionToken)) { return StaticCredentialsProvider.create( @@ -154,6 +180,26 @@ public AwsCredentialsProvider credentialsProvider( return DefaultCredentialsProvider.builder().build(); } + /** + * Configure RetryMode + * to ADAPTIVE_V2 for AWS clients + * + *

Sample usage: + * + *

+   *   KmsClient.builder().applyMutation(awsClientProperties::applyRetryConfigurations)
+   * 
+ */ + public void applyRetryConfigurations(T builder) { + ClientOverrideConfiguration.Builder configBuilder = + null != builder.overrideConfiguration() + ? builder.overrideConfiguration().toBuilder() + : ClientOverrideConfiguration.builder(); + + builder.overrideConfiguration(configBuilder.retryStrategy(RetryMode.ADAPTIVE_V2).build()); + } + private AwsCredentialsProvider credentialsProvider(String credentialsProviderClass) { Class providerClass; try { 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 5d37470066d2..aaab2c2bc769 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 @@ -96,6 +96,7 @@ public KmsClient kms() { if (isTableRegisteredWithLakeFormation()) { return KmsClient.builder() .applyMutation(httpClientProperties()::applyHttpClientConfigurations) + .applyMutation(awsClientProperties()::applyRetryConfigurations) .credentialsProvider( new LakeFormationCredentialsProvider(lakeFormation(), buildTableArn())) .region(Region.of(region())) 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 11a5ce02247f..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(); 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 3a43880f31ed..8d97b9d1bf20 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 @@ -225,6 +225,13 @@ public class S3FileIOProperties implements Serializable { */ public static final String SESSION_TOKEN = "s3.session-token"; + /** + * Configure the expiration time in millis of the static session token used to access S3FileIO. + * This expiration time is currently only used in {@link VendedCredentialsProvider} for refreshing + * vended credentials. + */ + static final String SESSION_TOKEN_EXPIRES_AT_MS = "s3.session-token-expires-at-ms"; + /** * Enable to make S3FileIO, to make cross-region call to the region specified in the ARN of an * access point. @@ -428,6 +435,25 @@ public class S3FileIOProperties implements Serializable { 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; @@ -462,6 +488,8 @@ public class S3FileIOProperties implements Serializable { private int s3RetryNumRetries; private long s3RetryMinWaitMs; private long s3RetryMaxWaitMs; + + private boolean s3DirectoryBucketListPrefixAsDirectory; private final Map allProperties; public S3FileIOProperties() { @@ -498,6 +526,8 @@ public S3FileIOProperties() { 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( @@ -605,6 +635,11 @@ public S3FileIOProperties(Map properties) { 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(), @@ -837,6 +872,15 @@ 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); } 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 74e602a27378..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 @@ -26,6 +26,7 @@ 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; @@ -35,6 +36,7 @@ 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; @@ -50,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; @@ -66,10 +71,18 @@ class S3InputStream extends SeekableInputStream implements RangeReadable { private int skipSize = 1024 * 1024; private RetryPolicy retryPolicy = RetryPolicy.builder() - .handle( - ImmutableList.of( - SSLException.class, SocketTimeoutException.class, SocketException.class)) - .onFailure(failure -> openStream(true)) + .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(); @@ -230,6 +243,11 @@ private void openStream(boolean closeQuietly) 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 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/main/java/org/apache/iceberg/aws/s3/VendedCredentialsProvider.java b/aws/src/main/java/org/apache/iceberg/aws/s3/VendedCredentialsProvider.java new file mode 100644 index 000000000000..e249d3ff1dec --- /dev/null +++ b/aws/src/main/java/org/apache/iceberg/aws/s3/VendedCredentialsProvider.java @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.aws.s3; + +import java.time.Instant; +import java.time.temporal.ChronoUnit; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.rest.ErrorHandlers; +import org.apache.iceberg.rest.HTTPClient; +import org.apache.iceberg.rest.RESTClient; +import org.apache.iceberg.rest.auth.OAuth2Properties; +import org.apache.iceberg.rest.auth.OAuth2Util; +import org.apache.iceberg.rest.credentials.Credential; +import org.apache.iceberg.rest.responses.LoadCredentialsResponse; +import software.amazon.awssdk.auth.credentials.AwsCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.auth.credentials.AwsSessionCredentials; +import software.amazon.awssdk.utils.IoUtils; +import software.amazon.awssdk.utils.SdkAutoCloseable; +import software.amazon.awssdk.utils.cache.CachedSupplier; +import software.amazon.awssdk.utils.cache.RefreshResult; + +public class VendedCredentialsProvider implements AwsCredentialsProvider, SdkAutoCloseable { + public static final String URI = "credentials.uri"; + private volatile HTTPClient client; + private final Map properties; + private final CachedSupplier credentialCache; + + private VendedCredentialsProvider(Map properties) { + Preconditions.checkArgument(null != properties, "Invalid properties: null"); + Preconditions.checkArgument(null != properties.get(URI), "Invalid URI: null"); + this.properties = properties; + this.credentialCache = + CachedSupplier.builder(this::refreshCredential) + .cachedValueName(VendedCredentialsProvider.class.getName()) + .build(); + } + + @Override + public AwsCredentials resolveCredentials() { + return credentialCache.get(); + } + + @Override + public void close() { + IoUtils.closeQuietly(client, null); + credentialCache.close(); + } + + public static VendedCredentialsProvider create(Map properties) { + return new VendedCredentialsProvider(properties); + } + + private RESTClient httpClient() { + if (null == client) { + synchronized (this) { + if (null == client) { + client = HTTPClient.builder(properties).uri(properties.get(URI)).build(); + } + } + } + + return client; + } + + private LoadCredentialsResponse fetchCredentials() { + return httpClient() + .get( + properties.get(URI), + null, + LoadCredentialsResponse.class, + OAuth2Util.authHeaders(properties.get(OAuth2Properties.TOKEN)), + ErrorHandlers.defaultErrorHandler()); + } + + private RefreshResult refreshCredential() { + LoadCredentialsResponse response = fetchCredentials(); + + List s3Credentials = + response.credentials().stream() + .filter(c -> c.prefix().startsWith("s3")) + .collect(Collectors.toList()); + + Preconditions.checkState(!s3Credentials.isEmpty(), "Invalid S3 Credentials: empty"); + Preconditions.checkState( + s3Credentials.size() == 1, "Invalid S3 Credentials: only one S3 credential should exist"); + + Credential s3Credential = s3Credentials.get(0); + checkCredential(s3Credential, S3FileIOProperties.ACCESS_KEY_ID); + checkCredential(s3Credential, S3FileIOProperties.SECRET_ACCESS_KEY); + checkCredential(s3Credential, S3FileIOProperties.SESSION_TOKEN); + checkCredential(s3Credential, S3FileIOProperties.SESSION_TOKEN_EXPIRES_AT_MS); + + String accessKeyId = s3Credential.config().get(S3FileIOProperties.ACCESS_KEY_ID); + String secretAccessKey = s3Credential.config().get(S3FileIOProperties.SECRET_ACCESS_KEY); + String sessionToken = s3Credential.config().get(S3FileIOProperties.SESSION_TOKEN); + String tokenExpiresAtMillis = + s3Credential.config().get(S3FileIOProperties.SESSION_TOKEN_EXPIRES_AT_MS); + Instant expiresAt = Instant.ofEpochMilli(Long.parseLong(tokenExpiresAtMillis)); + Instant prefetchAt = expiresAt.minus(5, ChronoUnit.MINUTES); + + return RefreshResult.builder( + (AwsCredentials) + AwsSessionCredentials.builder() + .accessKeyId(accessKeyId) + .secretAccessKey(secretAccessKey) + .sessionToken(sessionToken) + .expirationTime(expiresAt) + .build()) + .staleTime(expiresAt) + .prefetchTime(prefetchAt) + .build(); + } + + private void checkCredential(Credential credential, String property) { + Preconditions.checkState( + credential.config().containsKey(property), "Invalid S3 Credentials: %s not set", property); + } +} diff --git a/aws/src/test/java/org/apache/iceberg/aws/AwsClientPropertiesTest.java b/aws/src/test/java/org/apache/iceberg/aws/AwsClientPropertiesTest.java index c318538d9509..5cf9dd810c9f 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/AwsClientPropertiesTest.java +++ b/aws/src/test/java/org/apache/iceberg/aws/AwsClientPropertiesTest.java @@ -21,6 +21,8 @@ import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; +import org.apache.iceberg.aws.s3.VendedCredentialsProvider; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.junit.jupiter.api.Test; import org.mockito.ArgumentCaptor; @@ -29,6 +31,7 @@ import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; import software.amazon.awssdk.auth.credentials.AwsSessionCredentials; import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; +import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.s3.S3ClientBuilder; @@ -111,4 +114,30 @@ public void testSessionCredentialsConfiguration() { .as("The secret access key should be the same as the one set by tag SECRET_ACCESS_KEY") .isEqualTo("secret"); } + + @Test + public void refreshCredentialsEndpoint() { + AwsClientProperties awsClientProperties = + new AwsClientProperties( + ImmutableMap.of( + AwsClientProperties.REFRESH_CREDENTIALS_ENDPOINT, + "http://localhost:1234/v1/credentials")); + + assertThat(awsClientProperties.credentialsProvider("key", "secret", "token")) + .isInstanceOf(VendedCredentialsProvider.class); + } + + @Test + public void refreshCredentialsEndpointSetButRefreshDisabled() { + AwsClientProperties awsClientProperties = + new AwsClientProperties( + ImmutableMap.of( + AwsClientProperties.REFRESH_CREDENTIALS_ENABLED, + "false", + AwsClientProperties.REFRESH_CREDENTIALS_ENDPOINT, + "http://localhost:1234/v1/credentials")); + + assertThat(awsClientProperties.credentialsProvider("key", "secret", "token")) + .isInstanceOf(StaticCredentialsProvider.class); + } } diff --git a/aws/src/test/java/org/apache/iceberg/aws/TestS3FileIOProperties.java b/aws/src/test/java/org/apache/iceberg/aws/TestS3FileIOProperties.java index e2499e947617..58332d42588e 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/TestS3FileIOProperties.java +++ b/aws/src/test/java/org/apache/iceberg/aws/TestS3FileIOProperties.java @@ -312,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/glue/TestIcebergToGlueConverter.java b/aws/src/test/java/org/apache/iceberg/aws/glue/TestIcebergToGlueConverter.java index 1136ad63b410..edebfd3420e2 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/glue/TestIcebergToGlueConverter.java +++ b/aws/src/test/java/org/apache/iceberg/aws/glue/TestIcebergToGlueConverter.java @@ -238,7 +238,7 @@ public void testSetTableInputInformationWithRemovedColumns() { Schema newSchema = new Schema(Types.NestedField.required(1, "x", Types.StringType.get(), "comment1")); - tableMetadata = tableMetadata.updateSchema(newSchema, 3); + tableMetadata = tableMetadata.updateSchema(newSchema); IcebergToGlueConverter.setTableInputInformation(actualTableInputBuilder, tableMetadata); TableInput actualTableInput = actualTableInputBuilder.build(); diff --git a/aws/src/test/java/org/apache/iceberg/aws/kms/TestKmsClientProperties.java b/aws/src/test/java/org/apache/iceberg/aws/kms/TestKmsClientProperties.java new file mode 100644 index 000000000000..9d1fb8db51eb --- /dev/null +++ b/aws/src/test/java/org/apache/iceberg/aws/kms/TestKmsClientProperties.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.aws.kms; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.aws.AwsClientProperties; +import org.junit.jupiter.api.Test; +import software.amazon.awssdk.core.retry.RetryMode; +import software.amazon.awssdk.services.kms.KmsClient; +import software.amazon.awssdk.services.kms.KmsClientBuilder; + +public class TestKmsClientProperties { + @Test + public void testApplyRetryConfiguration() { + AwsClientProperties awsClientProperties = new AwsClientProperties(); + + KmsClientBuilder builder = KmsClient.builder(); + awsClientProperties.applyRetryConfigurations(builder); + RetryMode retryPolicy = builder.overrideConfiguration().retryMode().get(); + + assertThat(retryPolicy).as("retry mode should be ADAPTIVE_V2").isEqualTo(RetryMode.ADAPTIVE_V2); + } +} 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 index 08d14512cdc7..f98d1a3d4471 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/TestFlakyS3InputStream.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestFlakyS3InputStream.java @@ -18,6 +18,7 @@ */ 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; @@ -29,6 +30,7 @@ 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; @@ -49,10 +51,29 @@ 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 @@ -61,6 +82,7 @@ public void testReadWithFlakyStreamExhaustedRetries(IOException exception) { assertThatThrownBy(() -> testRead(flakyStreamClient(new AtomicInteger(5), exception))) .isInstanceOf(exception.getClass()) .hasMessage(exception.getMessage()); + assertThat(resetForRetryCounter.get()).isEqualTo(3); } @ParameterizedTest @@ -69,12 +91,14 @@ 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 @@ -83,6 +107,7 @@ public void testSeekWithFlakyStreamExhaustedRetries(IOException exception) { assertThatThrownBy(() -> testSeek(flakyStreamClient(new AtomicInteger(5), exception))) .isInstanceOf(exception.getClass()) .hasMessage(exception.getMessage()); + assertThat(resetForRetryCounter.get()).isEqualTo(3); } @ParameterizedTest @@ -91,6 +116,7 @@ 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() { 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 6caa42fb410b..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); } @@ -257,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. 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 0e3f8b2136a6..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 @@ -57,6 +56,10 @@ 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; @@ -64,7 +67,7 @@ protected void testRead(S3Client s3Client) throws Exception { writeS3Data(uri, data); - try (SeekableInputStream in = new S3InputStream(s3Client, uri)) { + try (SeekableInputStream in = newInputStream(s3Client, uri)) { int readSize = 1024; readAndCheck(in, in.getPos(), readSize, data, false); readAndCheck(in, in.getPos(), readSize, data, true); @@ -128,7 +131,7 @@ protected void testRangeRead(S3Client s3Client) throws Exception { writeS3Data(uri, expected); - try (RangeReadable in = new S3InputStream(s3Client, uri)) { + try (RangeReadable in = newInputStream(s3Client, uri)) { // first 1k position = 0; offset = 0; @@ -160,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) @@ -178,7 +181,7 @@ protected void testSeek(S3Client s3Client) throws Exception { writeS3Data(uri, expected); - try (SeekableInputStream in = new S3InputStream(s3Client, 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); 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/TestVendedCredentialsProvider.java b/aws/src/test/java/org/apache/iceberg/aws/s3/TestVendedCredentialsProvider.java new file mode 100644 index 000000000000..67cd1cb55241 --- /dev/null +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestVendedCredentialsProvider.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.aws.s3; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockserver.integration.ClientAndServer.startClientAndServer; +import static org.mockserver.model.HttpRequest.request; +import static org.mockserver.model.HttpResponse.response; + +import java.time.Instant; +import java.time.temporal.ChronoUnit; +import org.apache.iceberg.exceptions.RESTException; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.rest.HttpMethod; +import org.apache.iceberg.rest.credentials.Credential; +import org.apache.iceberg.rest.credentials.ImmutableCredential; +import org.apache.iceberg.rest.responses.ImmutableLoadCredentialsResponse; +import org.apache.iceberg.rest.responses.LoadCredentialsResponse; +import org.apache.iceberg.rest.responses.LoadCredentialsResponseParser; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockserver.integration.ClientAndServer; +import org.mockserver.model.HttpRequest; +import org.mockserver.model.HttpResponse; +import org.mockserver.verify.VerificationTimes; +import software.amazon.awssdk.auth.credentials.AwsCredentials; +import software.amazon.awssdk.auth.credentials.AwsSessionCredentials; + +public class TestVendedCredentialsProvider { + + private static final int PORT = 3232; + private static final String URI = String.format("http://127.0.0.1:%d/v1/credentials", PORT); + private static ClientAndServer mockServer; + + @BeforeAll + public static void beforeAll() { + mockServer = startClientAndServer(PORT); + } + + @AfterAll + public static void stopServer() { + mockServer.stop(); + } + + @BeforeEach + public void before() { + mockServer.reset(); + } + + @Test + public void invalidOrMissingUri() { + assertThatThrownBy(() -> VendedCredentialsProvider.create(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid properties: null"); + assertThatThrownBy(() -> VendedCredentialsProvider.create(ImmutableMap.of())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid URI: null"); + + try (VendedCredentialsProvider provider = + VendedCredentialsProvider.create( + ImmutableMap.of(VendedCredentialsProvider.URI, "invalid uri"))) { + assertThatThrownBy(provider::resolveCredentials) + .isInstanceOf(RESTException.class) + .hasMessageStartingWith("Failed to create request URI from base invalid uri"); + } + } + + @Test + public void noS3Credentials() { + HttpRequest mockRequest = request("/v1/credentials").withMethod(HttpMethod.GET.name()); + + HttpResponse mockResponse = + response( + LoadCredentialsResponseParser.toJson( + ImmutableLoadCredentialsResponse.builder().build())) + .withStatusCode(200); + mockServer.when(mockRequest).respond(mockResponse); + + try (VendedCredentialsProvider provider = + VendedCredentialsProvider.create(ImmutableMap.of(VendedCredentialsProvider.URI, URI))) { + assertThatThrownBy(provider::resolveCredentials) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Invalid S3 Credentials: empty"); + } + } + + @Test + public void accessKeyIdAndSecretAccessKeyWithoutToken() { + HttpRequest mockRequest = request("/v1/credentials").withMethod(HttpMethod.GET.name()); + LoadCredentialsResponse response = + ImmutableLoadCredentialsResponse.builder() + .addCredentials( + ImmutableCredential.builder() + .prefix("s3") + .config( + ImmutableMap.of( + S3FileIOProperties.ACCESS_KEY_ID, + "randomAccessKey", + S3FileIOProperties.SECRET_ACCESS_KEY, + "randomSecretAccessKey")) + .build()) + .build(); + + HttpResponse mockResponse = + response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); + mockServer.when(mockRequest).respond(mockResponse); + + try (VendedCredentialsProvider provider = + VendedCredentialsProvider.create(ImmutableMap.of(VendedCredentialsProvider.URI, URI))) { + assertThatThrownBy(provider::resolveCredentials) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Invalid S3 Credentials: s3.session-token not set"); + } + } + + @Test + public void expirationNotSet() { + HttpRequest mockRequest = request("/v1/credentials").withMethod(HttpMethod.GET.name()); + LoadCredentialsResponse response = + ImmutableLoadCredentialsResponse.builder() + .addCredentials( + ImmutableCredential.builder() + .prefix("s3") + .config( + ImmutableMap.of( + S3FileIOProperties.ACCESS_KEY_ID, + "randomAccessKey", + S3FileIOProperties.SECRET_ACCESS_KEY, + "randomSecretAccessKey", + S3FileIOProperties.SESSION_TOKEN, + "sessionToken")) + .build()) + .build(); + + HttpResponse mockResponse = + response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); + mockServer.when(mockRequest).respond(mockResponse); + + try (VendedCredentialsProvider provider = + VendedCredentialsProvider.create(ImmutableMap.of(VendedCredentialsProvider.URI, URI))) { + assertThatThrownBy(provider::resolveCredentials) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Invalid S3 Credentials: s3.session-token-expires-at-ms not set"); + } + } + + @Test + public void nonExpiredToken() { + HttpRequest mockRequest = request("/v1/credentials").withMethod(HttpMethod.GET.name()); + Credential credential = + ImmutableCredential.builder() + .prefix("s3") + .config( + ImmutableMap.of( + S3FileIOProperties.ACCESS_KEY_ID, + "randomAccessKey", + S3FileIOProperties.SECRET_ACCESS_KEY, + "randomSecretAccessKey", + S3FileIOProperties.SESSION_TOKEN, + "sessionToken", + S3FileIOProperties.SESSION_TOKEN_EXPIRES_AT_MS, + Long.toString(Instant.now().plus(1, ChronoUnit.HOURS).toEpochMilli()))) + .build(); + LoadCredentialsResponse response = + ImmutableLoadCredentialsResponse.builder().addCredentials(credential).build(); + + HttpResponse mockResponse = + response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); + mockServer.when(mockRequest).respond(mockResponse); + + try (VendedCredentialsProvider provider = + VendedCredentialsProvider.create(ImmutableMap.of(VendedCredentialsProvider.URI, URI))) { + AwsCredentials awsCredentials = provider.resolveCredentials(); + + verifyCredentials(awsCredentials, credential); + + for (int i = 0; i < 5; i++) { + // resolving credentials multiple times should not hit the credentials endpoint again + assertThat(provider.resolveCredentials()).isSameAs(awsCredentials); + } + } + + mockServer.verify(mockRequest, VerificationTimes.once()); + } + + @Test + public void expiredToken() { + HttpRequest mockRequest = request("/v1/credentials").withMethod(HttpMethod.GET.name()); + Credential credential = + ImmutableCredential.builder() + .prefix("s3") + .config( + ImmutableMap.of( + S3FileIOProperties.ACCESS_KEY_ID, + "randomAccessKey", + S3FileIOProperties.SECRET_ACCESS_KEY, + "randomSecretAccessKey", + S3FileIOProperties.SESSION_TOKEN, + "sessionToken", + S3FileIOProperties.SESSION_TOKEN_EXPIRES_AT_MS, + Long.toString(Instant.now().minus(1, ChronoUnit.MINUTES).toEpochMilli()))) + .build(); + LoadCredentialsResponse response = + ImmutableLoadCredentialsResponse.builder().addCredentials(credential).build(); + + HttpResponse mockResponse = + response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); + mockServer.when(mockRequest).respond(mockResponse); + + try (VendedCredentialsProvider provider = + VendedCredentialsProvider.create(ImmutableMap.of(VendedCredentialsProvider.URI, URI))) { + AwsCredentials awsCredentials = provider.resolveCredentials(); + verifyCredentials(awsCredentials, credential); + + // resolving credentials multiple times should hit the credentials endpoint again + AwsCredentials refreshedCredentials = provider.resolveCredentials(); + assertThat(refreshedCredentials).isNotSameAs(awsCredentials); + verifyCredentials(refreshedCredentials, credential); + } + + mockServer.verify(mockRequest, VerificationTimes.exactly(2)); + } + + @Test + public void multipleS3Credentials() { + HttpRequest mockRequest = request("/v1/credentials").withMethod(HttpMethod.GET.name()); + Credential credentialOne = + ImmutableCredential.builder() + .prefix("gcs") + .config( + ImmutableMap.of( + S3FileIOProperties.ACCESS_KEY_ID, + "randomAccessKey1", + S3FileIOProperties.SECRET_ACCESS_KEY, + "randomSecretAccessKey1", + S3FileIOProperties.SESSION_TOKEN, + "sessionToken1", + S3FileIOProperties.SESSION_TOKEN_EXPIRES_AT_MS, + Long.toString(Instant.now().plus(1, ChronoUnit.HOURS).toEpochMilli()))) + .build(); + Credential credentialTwo = + ImmutableCredential.builder() + .prefix("s3://custom-uri/longest-prefix") + .config( + ImmutableMap.of( + S3FileIOProperties.ACCESS_KEY_ID, + "randomAccessKey2", + S3FileIOProperties.SECRET_ACCESS_KEY, + "randomSecretAccessKey2", + S3FileIOProperties.SESSION_TOKEN, + "sessionToken2", + S3FileIOProperties.SESSION_TOKEN_EXPIRES_AT_MS, + Long.toString(Instant.now().plus(2, ChronoUnit.HOURS).toEpochMilli()))) + .build(); + Credential credentialThree = + ImmutableCredential.builder() + .prefix("s3://custom-uri/long") + .config( + ImmutableMap.of( + S3FileIOProperties.ACCESS_KEY_ID, + "randomAccessKey3", + S3FileIOProperties.SECRET_ACCESS_KEY, + "randomSecretAccessKey3", + S3FileIOProperties.SESSION_TOKEN, + "sessionToken3", + S3FileIOProperties.SESSION_TOKEN_EXPIRES_AT_MS, + Long.toString(Instant.now().plus(3, ChronoUnit.HOURS).toEpochMilli()))) + .build(); + LoadCredentialsResponse response = + ImmutableLoadCredentialsResponse.builder() + .addCredentials(credentialOne, credentialTwo, credentialThree) + .build(); + + HttpResponse mockResponse = + response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); + mockServer.when(mockRequest).respond(mockResponse); + + try (VendedCredentialsProvider provider = + VendedCredentialsProvider.create(ImmutableMap.of(VendedCredentialsProvider.URI, URI))) { + assertThatThrownBy(provider::resolveCredentials) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Invalid S3 Credentials: only one S3 credential should exist"); + } + } + + private void verifyCredentials(AwsCredentials awsCredentials, Credential credential) { + assertThat(awsCredentials).isInstanceOf(AwsSessionCredentials.class); + AwsSessionCredentials creds = (AwsSessionCredentials) awsCredentials; + + assertThat(creds.accessKeyId()) + .isEqualTo(credential.config().get(S3FileIOProperties.ACCESS_KEY_ID)); + assertThat(creds.secretAccessKey()) + .isEqualTo(credential.config().get(S3FileIOProperties.SECRET_ACCESS_KEY)); + assertThat(creds.sessionToken()) + .isEqualTo(credential.config().get(S3FileIOProperties.SESSION_TOKEN)); + assertThat(creds.expirationTime()) + .isPresent() + .get() + .extracting(Instant::toEpochMilli) + .isEqualTo( + Long.parseLong( + credential.config().get(S3FileIOProperties.SESSION_TOKEN_EXPIRES_AT_MS))); + } +} 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/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSLocation.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSLocation.java index e73093512b82..5af590628fe8 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSLocation.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSLocation.java @@ -25,14 +25,16 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; /** - * This class represents a fully qualified location in Azure expressed as a URI. + * This class represents a fully qualified location to a file or directory in Azure Data Lake + * Storage Gen2 storage. * - *

Locations follow the conventions used by Hadoop's Azure support, i.e. + *

Locations follow a URI like structure to identify resources * *

{@code abfs[s]://[@]/}
* - *

See Hadoop Azure - * Support + *

See Azure + * Data Lake Storage URI */ class ADLSLocation { private static final Pattern URI_PATTERN = Pattern.compile("^abfss?://([^/?#]+)(.*)?$"); @@ -64,8 +66,7 @@ class ADLSLocation { } String uriPath = matcher.group(2); - uriPath = uriPath == null ? "" : uriPath.startsWith("/") ? uriPath.substring(1) : uriPath; - this.path = uriPath.split("\\?", -1)[0].split("#", -1)[0]; + this.path = uriPath == null ? "" : uriPath.startsWith("/") ? uriPath.substring(1) : uriPath; } /** Returns Azure storage account. */ diff --git a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/ADLSLocationTest.java b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/ADLSLocationTest.java index 867b54b4c7e3..403886f4b28e 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/ADLSLocationTest.java +++ b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/ADLSLocationTest.java @@ -82,23 +82,11 @@ public void testNoPath() { assertThat(location.path()).isEqualTo(""); } - @Test - public void testQueryAndFragment() { - String p1 = "abfs://container@account.dfs.core.windows.net/path/to/file?query=foo#123"; - ADLSLocation location = new ADLSLocation(p1); - - assertThat(location.storageAccount()).isEqualTo("account.dfs.core.windows.net"); - assertThat(location.container().get()).isEqualTo("container"); - assertThat(location.path()).isEqualTo("path/to/file"); - } - - @Test - public void testQueryAndFragmentNoPath() { - String p1 = "abfs://container@account.dfs.core.windows.net?query=foo#123"; - ADLSLocation location = new ADLSLocation(p1); - - assertThat(location.storageAccount()).isEqualTo("account.dfs.core.windows.net"); - assertThat(location.container().get()).isEqualTo("container"); - assertThat(location.path()).isEqualTo(""); + @ParameterizedTest + @ValueSource(strings = {"file?.txt", "file%3F.txt"}) + public void testQuestionMarkInFileName(String path) { + String fullPath = String.format("abfs://container@account.dfs.core.windows.net/%s", path); + ADLSLocation location = new ADLSLocation(fullPath); + assertThat(location.path()).contains(path); } } 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/build.gradle b/build.gradle index e66ea793bbf7..46fd1e3bb3f9 100644 --- a/build.gradle +++ b/build.gradle @@ -26,8 +26,8 @@ buildscript { gradlePluginPortal() } dependencies { - classpath 'com.gradleup.shadow:shadow-gradle-plugin:8.3.3' - classpath 'com.palantir.baseline:gradle-baseline-java:5.69.0' + classpath 'com.gradleup.shadow:shadow-gradle-plugin:8.3.5' + 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' @@ -135,7 +135,7 @@ subprojects { revapi { oldGroup = project.group oldName = project.name - oldVersion = "1.6.0" + oldVersion = "1.7.0" } tasks.register('showDeprecationRulesOnRevApiFailure') { @@ -347,7 +347,6 @@ 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' @@ -438,16 +437,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' - } } } @@ -495,20 +484,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 { @@ -656,6 +643,7 @@ project(':iceberg-gcp') { testImplementation "com.google.cloud:google-cloud-nio" testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') + testImplementation project(path: ':iceberg-core', configuration: 'testArtifacts') testImplementation(libs.hadoop2.common) { exclude group: 'org.apache.avro', module: 'avro' @@ -664,6 +652,8 @@ project(':iceberg-gcp') { exclude group: 'com.google.code.gson', module: 'gson' } testImplementation libs.esotericsoftware.kryo + testImplementation libs.mockserver.netty + testImplementation libs.mockserver.client.java } } @@ -847,39 +837,6 @@ project(':iceberg-arrow') { } } -project(':iceberg-pig') { - test { - useJUnitPlatform() - } - - dependencies { - implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') - api project(':iceberg-api') - implementation project(':iceberg-common') - implementation project(':iceberg-core') - implementation project(':iceberg-parquet') - - implementation(libs.parquet.avro) { - exclude group: 'org.apache.avro', module: 'avro' - // already shaded by Parquet - exclude group: 'it.unimi.dsi' - exclude group: 'org.codehaus.jackson' - } - - compileOnly(libs.pig) { - exclude group: "junit", module: "junit" - } - compileOnly(libs.hadoop2.mapreduce.client.core) - compileOnly(libs.hadoop2.client) { - exclude group: 'org.apache.avro', module: 'avro' - } - - testImplementation(libs.hadoop2.minicluster) { - exclude group: 'org.apache.avro', module: 'avro' - } - } -} - project(':iceberg-nessie') { test { useJUnitPlatform() @@ -956,6 +913,9 @@ project(':iceberg-snowflake') { project(':iceberg-open-api') { apply plugin: 'java-test-fixtures' + apply plugin: 'com.gradleup.shadow' + + build.dependsOn shadowJar dependencies { testImplementation project(':iceberg-api') @@ -975,16 +935,45 @@ project(':iceberg-open-api') { 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.hadoop3.common) { + exclude group: 'org.slf4j' + exclude group: 'ch.qos.reload4j' + exclude group: 'org.apache.avro', module: 'avro' + exclude group: 'com.google.guava' + exclude group: 'com.google.protobuf' + exclude group: 'org.apache.curator' + exclude group: 'org.apache.zookeeper' + exclude group: 'org.apache.kerby' + exclude group: 'org.apache.hadoop.thirdparty', module: 'hadoop-shaded-protobuf_3_7' + exclude group: 'org.eclipse.jetty' + exclude group: 'com.google.re2j', module: 're2j' + exclude group: 'com.google.code.gson', module: 'gson' + exclude group: 'com.jcraft', module: 'jsch' + exclude group: 'com.google.code.findbugs', module: 'jsr305' + exclude group: 'io.dropwizard.metrics', module: 'metrics-core' + exclude group: 'dnsjava', module: 'dnsjava' + exclude group: 'org.xerial.snappy', module: 'snappy-java' + exclude group: 'commons-cli', module: 'commons-cli' + exclude group: 'com.github.pjfanning', module: 'jersey-json' + } + testFixturesImplementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') + testFixturesImplementation libs.junit.jupiter + + testFixturesImplementation libs.slf4j.api + testFixturesImplementation libs.slf4j.simple testFixturesImplementation libs.jetty.servlet testFixturesImplementation libs.jetty.server testFixturesImplementation libs.sqlite.jdbc testFixturesCompileOnly libs.apiguardian + + testFixturesRuntimeOnly project(':iceberg-aws-bundle') + testFixturesRuntimeOnly project(':iceberg-azure-bundle') + testFixturesRuntimeOnly project(':iceberg-gcp-bundle') } test { @@ -1008,6 +997,28 @@ project(':iceberg-open-api') { recommend.set(true) } check.dependsOn('validateRESTCatalogSpec') + + shadowJar { + archiveBaseName.set("iceberg-open-api-test-fixtures-runtime") + archiveClassifier.set(null) + configurations = [project.configurations.testFixturesRuntimeClasspath] + from sourceSets.testFixtures.output + zip64 true + + // include the LICENSE and NOTICE files for the runtime Jar + from(projectDir) { + include 'LICENSE' + include 'NOTICE' + } + + manifest { + attributes 'Main-Class': 'org.apache.iceberg.rest.RESTCatalogServer' + } + } + + jar { + enabled = false + } } @Memoized diff --git a/core/src/jmh/java/org/apache/iceberg/ManifestWriteBenchmark.java b/core/src/jmh/java/org/apache/iceberg/ManifestWriteBenchmark.java index c175248241e9..349c8e5d4fa2 100644 --- a/core/src/jmh/java/org/apache/iceberg/ManifestWriteBenchmark.java +++ b/core/src/jmh/java/org/apache/iceberg/ManifestWriteBenchmark.java @@ -29,7 +29,6 @@ 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; @@ -96,7 +95,8 @@ public int getFormatVersion() { @Benchmark @Threads(1) public void writeManifestFile(BenchmarkState state) throws IOException { - this.baseDir = Files.createTempDir().getAbsolutePath(); + this.baseDir = + java.nio.file.Files.createTempDirectory("benchmark-").toAbsolutePath().toString(); this.manifestListFile = String.format("%s/%s.avro", baseDir, UUID.randomUUID()); try (ManifestListWriter listWriter = diff --git a/core/src/jmh/java/org/apache/iceberg/ReplaceDeleteFilesBenchmark.java b/core/src/jmh/java/org/apache/iceberg/ReplaceDeleteFilesBenchmark.java index a899b870a90c..1db327da8c8c 100644 --- a/core/src/jmh/java/org/apache/iceberg/ReplaceDeleteFilesBenchmark.java +++ b/core/src/jmh/java/org/apache/iceberg/ReplaceDeleteFilesBenchmark.java @@ -20,10 +20,13 @@ import static org.apache.iceberg.types.Types.NestedField.required; +import java.io.IOException; import java.util.List; +import java.util.Map; 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.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; @@ -70,14 +73,17 @@ public class ReplaceDeleteFilesBenchmark { private static final HadoopTables TABLES = new HadoopTables(); private Table table; - private List deleteFiles; + private List deleteFilesToReplace; private List pendingDeleteFiles; - @Param({"50000", "100000", "500000", "1000000", "2500000"}) + @Param({"50000", "100000", "500000", "1000000", "2000000"}) private int numFiles; + @Param({"5", "25", "50", "100"}) + private int percentDeleteFilesReplaced; + @Setup - public void setupBenchmark() { + public void setupBenchmark() throws IOException { initTable(); initFiles(); } @@ -90,10 +96,13 @@ public void tearDownBenchmark() { @Benchmark @Threads(1) public void replaceDeleteFiles() { + Snapshot currentSnapshot = table.currentSnapshot(); RowDelta rowDelta = table.newRowDelta(); - deleteFiles.forEach(rowDelta::removeDeletes); + rowDelta.validateFromSnapshot(currentSnapshot.snapshotId()); + deleteFilesToReplace.forEach(rowDelta::removeDeletes); pendingDeleteFiles.forEach(rowDelta::addDeletes); rowDelta.commit(); + table.manageSnapshots().rollbackTo(currentSnapshot.snapshotId()).commit(); } private void initTable() { @@ -104,27 +113,44 @@ private void dropTable() { TABLES.dropTable(TABLE_IDENT); } - private void initFiles() { - List generatedDeleteFiles = Lists.newArrayListWithExpectedSize(numFiles); + private void initFiles() throws IOException { List generatedPendingDeleteFiles = Lists.newArrayListWithExpectedSize(numFiles); - + int numDeleteFilesToReplace = (int) Math.ceil(numFiles * (percentDeleteFilesReplaced / 100.0)); + Map filesToReplace = + Maps.newHashMapWithExpectedSize(numDeleteFilesToReplace); 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); + if (numDeleteFilesToReplace > 0) { + filesToReplace.put(deleteFile.location(), deleteFile); + DeleteFile pendingDeleteFile = + FileGenerationUtil.generatePositionDeleteFile(table, dataFile); + generatedPendingDeleteFiles.add(pendingDeleteFile); + numDeleteFilesToReplace--; + } } rowDelta.commit(); - this.deleteFiles = generatedDeleteFiles; + List deleteFilesReadFromManifests = Lists.newArrayList(); + for (ManifestFile deleteManifest : table.currentSnapshot().deleteManifests(table.io())) { + try (ManifestReader manifestReader = + ManifestFiles.readDeleteManifest(deleteManifest, table.io(), table.specs())) { + manifestReader + .iterator() + .forEachRemaining( + file -> { + if (filesToReplace.containsKey(file.location())) { + deleteFilesReadFromManifests.add(file); + } + }); + } + } + this.pendingDeleteFiles = generatedPendingDeleteFiles; + this.deleteFilesToReplace = deleteFilesReadFromManifests; } } 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/main/java/org/apache/iceberg/BaseChangelogContentScanTask.java b/core/src/main/java/org/apache/iceberg/BaseChangelogContentScanTask.java index 64c8cae08019..924148214ba0 100644 --- a/core/src/main/java/org/apache/iceberg/BaseChangelogContentScanTask.java +++ b/core/src/main/java/org/apache/iceberg/BaseChangelogContentScanTask.java @@ -56,7 +56,7 @@ public String toString() { return MoreObjects.toStringHelper(this) .add("change_ordinal", changeOrdinal) .add("commit_snapshot_id", commitSnapshotId) - .add("file", file().path()) + .add("file", file().location()) .add("partition_data", file().partition()) .add("residual", residual()) .toString(); @@ -142,7 +142,7 @@ public String toString() { return MoreObjects.toStringHelper(this) .add("change_ordinal", changeOrdinal()) .add("commit_snapshot_id", commitSnapshotId()) - .add("file", file().path()) + .add("file", file().location()) .add("partition_data", file().partition()) .add("offset", offset) .add("length", length) diff --git a/core/src/main/java/org/apache/iceberg/BaseContentScanTask.java b/core/src/main/java/org/apache/iceberg/BaseContentScanTask.java index 8d38a48309c6..53c2875a7043 100644 --- a/core/src/main/java/org/apache/iceberg/BaseContentScanTask.java +++ b/core/src/main/java/org/apache/iceberg/BaseContentScanTask.java @@ -23,6 +23,7 @@ import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.util.ArrayUtil; +import org.apache.iceberg.util.ScanTaskUtil; abstract class BaseContentScanTask, F extends ContentFile> implements ContentScanTask, SplittableScanTask { @@ -82,7 +83,7 @@ public long start() { @Override public long length() { - return file.fileSizeInBytes(); + return ScanTaskUtil.contentSizeInBytes(file); } @Override @@ -116,7 +117,7 @@ public Iterable split(long targetSplitSize) { @Override public String toString() { return MoreObjects.toStringHelper(this) - .add("file", file().path()) + .add("file", file().location()) .add("partition_data", file().partition()) .add("residual", residual()) .toString(); diff --git a/core/src/main/java/org/apache/iceberg/BaseFile.java b/core/src/main/java/org/apache/iceberg/BaseFile.java index 8f84eb5737b9..e9724637dfa3 100644 --- a/core/src/main/java/org/apache/iceberg/BaseFile.java +++ b/core/src/main/java/org/apache/iceberg/BaseFile.java @@ -80,6 +80,9 @@ public PartitionData copy() { private int[] equalityIds = null; private byte[] keyMetadata = null; private Integer sortOrderId; + private String referencedDataFile = null; + private Long contentOffset = null; + private Long contentSizeInBytes = null; // cached schema private transient Schema avroSchema = null; @@ -108,6 +111,9 @@ public PartitionData copy() { DataFile.SPLIT_OFFSETS, DataFile.EQUALITY_IDS, DataFile.SORT_ORDER_ID, + DataFile.REFERENCED_DATA_FILE, + DataFile.CONTENT_OFFSET, + DataFile.CONTENT_SIZE, MetadataColumns.ROW_POSITION); /** Used by Avro reflection to instantiate this class when reading manifest files. */ @@ -149,7 +155,10 @@ public PartitionData copy() { List splitOffsets, int[] equalityFieldIds, Integer sortOrderId, - ByteBuffer keyMetadata) { + ByteBuffer keyMetadata, + String referencedDataFile, + Long contentOffset, + Long contentSizeInBytes) { super(BASE_TYPE.fields().size()); this.partitionSpecId = specId; this.content = content; @@ -178,6 +187,9 @@ public PartitionData copy() { this.equalityIds = equalityFieldIds; this.sortOrderId = sortOrderId; this.keyMetadata = ByteBuffers.toByteArray(keyMetadata); + this.referencedDataFile = referencedDataFile; + this.contentOffset = contentOffset; + this.contentSizeInBytes = contentSizeInBytes; } /** @@ -230,6 +242,9 @@ public PartitionData copy() { this.sortOrderId = toCopy.sortOrderId; this.dataSequenceNumber = toCopy.dataSequenceNumber; this.fileSequenceNumber = toCopy.fileSequenceNumber; + this.referencedDataFile = toCopy.referencedDataFile; + this.contentOffset = toCopy.contentOffset; + this.contentSizeInBytes = toCopy.contentSizeInBytes; } /** Constructor for Java serialization. */ @@ -339,6 +354,15 @@ protected void internalSet(int pos, T value) { this.sortOrderId = (Integer) value; return; case 17: + this.referencedDataFile = value != null ? value.toString() : null; + return; + case 18: + this.contentOffset = (Long) value; + return; + case 19: + this.contentSizeInBytes = (Long) value; + return; + case 20: this.fileOrdinal = (long) value; return; default: @@ -388,6 +412,12 @@ private Object getByPos(int basePos) { case 16: return sortOrderId; case 17: + return referencedDataFile; + case 18: + return contentOffset; + case 19: + return contentSizeInBytes; + case 20: return fileOrdinal; default: throw new UnsupportedOperationException("Unknown field ordinal: " + basePos); @@ -514,6 +544,18 @@ public Integer sortOrderId() { return sortOrderId; } + public String referencedDataFile() { + return referencedDataFile; + } + + public Long contentOffset() { + return contentOffset; + } + + public Long contentSizeInBytes() { + return contentSizeInBytes; + } + private static Map copyMap(Map map, Set keys) { return keys == null ? SerializableMap.copyOf(map) : SerializableMap.filteredCopyOf(map, keys); } @@ -565,6 +607,9 @@ public String toString() { .add("sort_order_id", sortOrderId) .add("data_sequence_number", dataSequenceNumber == null ? "null" : dataSequenceNumber) .add("file_sequence_number", fileSequenceNumber == null ? "null" : fileSequenceNumber) + .add("referenced_data_file", referencedDataFile == null ? "null" : referencedDataFile) + .add("content_offset", contentOffset == null ? "null" : contentOffset) + .add("content_size_in_bytes", contentSizeInBytes == null ? "null" : contentSizeInBytes) .toString(); } } diff --git a/core/src/main/java/org/apache/iceberg/BaseFileScanTask.java b/core/src/main/java/org/apache/iceberg/BaseFileScanTask.java index 2469395021d4..2cc406444552 100644 --- a/core/src/main/java/org/apache/iceberg/BaseFileScanTask.java +++ b/core/src/main/java/org/apache/iceberg/BaseFileScanTask.java @@ -23,6 +23,7 @@ import org.apache.iceberg.expressions.ResidualEvaluator; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.util.ScanTaskUtil; public class BaseFileScanTask extends BaseContentScanTask implements FileScanTask { @@ -79,7 +80,7 @@ private long deletesSizeBytes() { if (deletesSizeBytes == 0L && deletes.length > 0) { long size = 0L; for (DeleteFile deleteFile : deletes) { - size += deleteFile.fileSizeInBytes(); + size += ScanTaskUtil.contentSizeInBytes(deleteFile); } this.deletesSizeBytes = size; } @@ -175,16 +176,14 @@ public boolean canMerge(ScanTask other) { @Override public SplitScanTask merge(ScanTask other) { SplitScanTask that = (SplitScanTask) other; + // don't use deletesSizeBytes() here so that deletesSizeBytes is only calculated once after + // merging rather than for each task before merging return new SplitScanTask(offset, len + that.length(), fileScanTask, deletesSizeBytes); } private long deletesSizeBytes() { if (deletesSizeBytes == 0L && fileScanTask.filesCount() > 1) { - long size = 0L; - for (DeleteFile deleteFile : fileScanTask.deletes()) { - size += deleteFile.fileSizeInBytes(); - } - this.deletesSizeBytes = size; + this.deletesSizeBytes = ScanTaskUtil.contentSizeInBytes(fileScanTask.deletes()); } return deletesSizeBytes; diff --git a/core/src/main/java/org/apache/iceberg/BaseFilesTable.java b/core/src/main/java/org/apache/iceberg/BaseFilesTable.java index b71744f739c7..4dff19b87990 100644 --- a/core/src/main/java/org/apache/iceberg/BaseFilesTable.java +++ b/core/src/main/java/org/apache/iceberg/BaseFilesTable.java @@ -181,15 +181,7 @@ public long estimatedRowsCount() { } private CloseableIterable> files(Schema fileProjection) { - switch (manifest.content()) { - case DATA: - return ManifestFiles.read(manifest, io, specsById).project(fileProjection); - case DELETES: - return ManifestFiles.readDeleteManifest(manifest, io, specsById).project(fileProjection); - default: - throw new IllegalArgumentException( - "Unsupported manifest content type:" + manifest.content()); - } + return ManifestFiles.open(manifest, io, specsById).project(fileProjection); } /** diff --git a/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java b/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java index e794b3121dc3..29068df380a9 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java +++ b/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java @@ -113,7 +113,7 @@ private Table loadMetadataTable(TableIdentifier identifier) { } } - private boolean isValidMetadataIdentifier(TableIdentifier identifier) { + protected boolean isValidMetadataIdentifier(TableIdentifier identifier) { return MetadataTableType.from(identifier.name()) != null && isValidIdentifier(TableIdentifier.of(identifier.namespace().levels())); } @@ -217,7 +217,8 @@ public Transaction createTransaction() { tableProperties.putAll(tableOverrideProperties()); TableMetadata metadata = TableMetadata.newTableMetadata(schema, spec, sortOrder, baseLocation, tableProperties); - return Transactions.createTableTransaction(identifier.toString(), ops, metadata); + return Transactions.createTableTransaction( + identifier.toString(), ops, metadata, metricsReporter()); } @Override @@ -249,9 +250,11 @@ private Transaction newReplaceTableTransaction(boolean orCreate) { } if (orCreate) { - return Transactions.createOrReplaceTableTransaction(identifier.toString(), ops, metadata); + return Transactions.createOrReplaceTableTransaction( + identifier.toString(), ops, metadata, metricsReporter()); } else { - return Transactions.replaceTableTransaction(identifier.toString(), ops, metadata); + return Transactions.replaceTableTransaction( + identifier.toString(), ops, metadata, metricsReporter()); } } diff --git a/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java b/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java index 53f3250dc95c..dbab9e813966 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java @@ -19,7 +19,6 @@ package org.apache.iceberg; import java.util.Locale; -import java.util.Set; import java.util.UUID; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; @@ -32,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; @@ -127,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( @@ -354,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 16fbc0dd1ebc..3bf44244862e 100644 --- a/core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java +++ b/core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java @@ -141,7 +141,7 @@ protected void validate(TableMetadata base, Snapshot parent) { && (strict.eval(file.partition()) || metrics.eval(file)), "Cannot append file with rows that do not match filter: %s: %s", rowFilter, - file.path()); + file.location()); } } diff --git a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java index dce6d4a995bd..ef578a82f3fa 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java +++ b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java @@ -52,7 +52,6 @@ public class BaseRewriteManifests extends SnapshotProducer private static final String REPLACED_MANIFESTS_COUNT = "manifests-replaced"; private static final String PROCESSED_ENTRY_COUNT = "entries-processed"; - private final TableOperations ops; private final Map specsById; private final long manifestTargetSizeBytes; @@ -74,10 +73,10 @@ public class BaseRewriteManifests extends SnapshotProducer BaseRewriteManifests(TableOperations ops) { super(ops); - this.ops = ops; - this.specsById = ops.current().specsById(); + this.specsById = ops().current().specsById(); this.manifestTargetSizeBytes = - ops.current() + ops() + .current() .propertyAsLong(MANIFEST_TARGET_SIZE_BYTES, MANIFEST_TARGET_SIZE_BYTES_DEFAULT); } @@ -153,8 +152,8 @@ public RewriteManifests addManifest(ManifestFile manifest) { } private ManifestFile copyManifest(ManifestFile manifest) { - TableMetadata current = ops.current(); - InputFile toCopy = ops.io().newInputFile(manifest); + TableMetadata current = ops().current(); + InputFile toCopy = ops().io().newInputFile(manifest); EncryptedOutputFile newFile = newManifestOutputFile(); return ManifestFiles.copyRewriteManifest( current.formatVersion(), @@ -168,7 +167,7 @@ private ManifestFile copyManifest(ManifestFile manifest) { @Override public List apply(TableMetadata base, Snapshot snapshot) { - List currentManifests = base.currentSnapshot().allManifests(ops.io()); + List currentManifests = base.currentSnapshot().allManifests(ops().io()); Set currentManifestSet = ImmutableSet.copyOf(currentManifests); validateDeletedManifests(currentManifestSet, base.currentSnapshot().snapshotId()); @@ -246,7 +245,7 @@ private void performRewrite(List currentManifests) { } else { rewrittenManifests.add(manifest); try (ManifestReader reader = - ManifestFiles.read(manifest, ops.io(), ops.current().specsById()) + ManifestFiles.read(manifest, ops().io(), ops().current().specsById()) .select(Collections.singletonList("*"))) { reader .liveEntries() diff --git a/core/src/main/java/org/apache/iceberg/BaseRowDelta.java b/core/src/main/java/org/apache/iceberg/BaseRowDelta.java index 85c2269ee526..372fc5367f08 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRowDelta.java +++ b/core/src/main/java/org/apache/iceberg/BaseRowDelta.java @@ -139,6 +139,8 @@ protected void validate(TableMetadata base, Snapshot parent) { if (validateNewDeleteFiles) { validateNoNewDeleteFiles(base, startingSnapshotId, conflictDetectionFilter, parent); } + + validateAddedDVs(base, startingSnapshotId, conflictDetectionFilter, parent); } } } diff --git a/core/src/main/java/org/apache/iceberg/BaseScan.java b/core/src/main/java/org/apache/iceberg/BaseScan.java index 804df01d31ba..618b2e95f29f 100644 --- a/core/src/main/java/org/apache/iceberg/BaseScan.java +++ b/core/src/main/java/org/apache/iceberg/BaseScan.java @@ -77,6 +77,9 @@ abstract class BaseScan> "partition", "key_metadata", "split_offsets", + "referenced_data_file", + "content_offset", + "content_size_in_bytes", "equality_ids"); protected static final List DELETE_SCAN_WITH_STATS_COLUMNS = diff --git a/core/src/main/java/org/apache/iceberg/BaseSnapshot.java b/core/src/main/java/org/apache/iceberg/BaseSnapshot.java index a3c4fc8738cd..58dec570d1fb 100644 --- a/core/src/main/java/org/apache/iceberg/BaseSnapshot.java +++ b/core/src/main/java/org/apache/iceberg/BaseSnapshot.java @@ -138,7 +138,8 @@ private void cacheManifests(FileIO fileIO) { allManifests = Lists.transform( Arrays.asList(v1ManifestLocations), - location -> new GenericManifestFile(fileIO.newInputFile(location), 0)); + location -> + new GenericManifestFile(fileIO.newInputFile(location), 0, this.snapshotId)); } if (allManifests == null) { diff --git a/core/src/main/java/org/apache/iceberg/CatalogUtil.java b/core/src/main/java/org/apache/iceberg/CatalogUtil.java index 70b10cbaeb62..37d9ad86e16d 100644 --- a/core/src/main/java/org/apache/iceberg/CatalogUtil.java +++ b/core/src/main/java/org/apache/iceberg/CatalogUtil.java @@ -175,7 +175,7 @@ private static void deleteFiles(FileIO io, Set allManifests) { for (ManifestEntry entry : reader.entries()) { // intern the file path because the weak key map uses identity (==) instead of // equals - String path = entry.file().path().toString().intern(); + String path = entry.file().location().intern(); Boolean alreadyDeleted = deletedFiles.putIfAbsent(path, true); if (alreadyDeleted == null || !alreadyDeleted) { pathsToDelete.add(path); @@ -515,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/ContentFileParser.java b/core/src/main/java/org/apache/iceberg/ContentFileParser.java index dd08c5c69e7d..1be06cb42602 100644 --- a/core/src/main/java/org/apache/iceberg/ContentFileParser.java +++ b/core/src/main/java/org/apache/iceberg/ContentFileParser.java @@ -45,6 +45,9 @@ class ContentFileParser { private static final String SPLIT_OFFSETS = "split-offsets"; private static final String EQUALITY_IDS = "equality-ids"; private static final String SORT_ORDER_ID = "sort-order-id"; + private static final String REFERENCED_DATA_FILE = "referenced-data-file"; + private static final String CONTENT_OFFSET = "content-offset"; + private static final String CONTENT_SIZE = "content-size-in-bytes"; private ContentFileParser() {} @@ -80,7 +83,7 @@ static void toJson(ContentFile contentFile, PartitionSpec spec, JsonGenerator generator.writeNumberField(SPEC_ID, contentFile.specId()); generator.writeStringField(CONTENT, contentFile.content().name()); - generator.writeStringField(FILE_PATH, contentFile.path().toString()); + generator.writeStringField(FILE_PATH, contentFile.location()); generator.writeStringField(FILE_FORMAT, contentFile.format().name()); if (contentFile.partition() != null) { @@ -109,6 +112,22 @@ static void toJson(ContentFile contentFile, PartitionSpec spec, JsonGenerator generator.writeNumberField(SORT_ORDER_ID, contentFile.sortOrderId()); } + if (contentFile instanceof DeleteFile) { + DeleteFile deleteFile = (DeleteFile) contentFile; + + if (deleteFile.referencedDataFile() != null) { + generator.writeStringField(REFERENCED_DATA_FILE, deleteFile.referencedDataFile()); + } + + if (deleteFile.contentOffset() != null) { + generator.writeNumberField(CONTENT_OFFSET, deleteFile.contentOffset()); + } + + if (deleteFile.contentSizeInBytes() != null) { + generator.writeNumberField(CONTENT_SIZE, deleteFile.contentSizeInBytes()); + } + } + generator.writeEndObject(); } @@ -145,6 +164,9 @@ static ContentFile fromJson(JsonNode jsonNode, PartitionSpec spec) { List splitOffsets = JsonUtil.getLongListOrNull(SPLIT_OFFSETS, jsonNode); int[] equalityFieldIds = JsonUtil.getIntArrayOrNull(EQUALITY_IDS, jsonNode); Integer sortOrderId = JsonUtil.getIntOrNull(SORT_ORDER_ID, jsonNode); + String referencedDataFile = JsonUtil.getStringOrNull(REFERENCED_DATA_FILE, jsonNode); + Long contentOffset = JsonUtil.getLongOrNull(CONTENT_OFFSET, jsonNode); + Long contentSizeInBytes = JsonUtil.getLongOrNull(CONTENT_SIZE, jsonNode); if (fileContent == FileContent.DATA) { return new GenericDataFile( @@ -169,7 +191,10 @@ static ContentFile fromJson(JsonNode jsonNode, PartitionSpec spec) { equalityFieldIds, sortOrderId, splitOffsets, - keyMetadata); + keyMetadata, + referencedDataFile, + contentOffset, + contentSizeInBytes); } } diff --git a/core/src/main/java/org/apache/iceberg/DataFiles.java b/core/src/main/java/org/apache/iceberg/DataFiles.java index 5c7c1a646cc9..0404f2da52b4 100644 --- a/core/src/main/java/org/apache/iceberg/DataFiles.java +++ b/core/src/main/java/org/apache/iceberg/DataFiles.java @@ -186,7 +186,7 @@ public Builder copy(DataFile toCopy) { specId == toCopy.specId(), "Cannot copy a DataFile with a different spec"); this.partitionData = copyPartitionData(spec, toCopy.partition(), partitionData); } - this.filePath = toCopy.path().toString(); + this.filePath = toCopy.location(); this.format = toCopy.format(); this.recordCount = toCopy.recordCount(); this.fileSizeInBytes = toCopy.fileSizeInBytes(); diff --git a/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java b/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java index 8444b91eecd4..ab7fec6fb151 100644 --- a/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java +++ b/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java @@ -33,6 +33,7 @@ import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutorService; import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.expressions.ManifestEvaluator; @@ -70,6 +71,7 @@ class DeleteFileIndex { private final PartitionMap eqDeletesByPartition; private final PartitionMap posDeletesByPartition; private final Map posDeletesByPath; + private final Map dvByPath; private final boolean hasEqDeletes; private final boolean hasPosDeletes; private final boolean isEmpty; @@ -78,13 +80,16 @@ private DeleteFileIndex( EqualityDeletes globalDeletes, PartitionMap eqDeletesByPartition, PartitionMap posDeletesByPartition, - Map posDeletesByPath) { + Map posDeletesByPath, + Map dvByPath) { this.globalDeletes = globalDeletes; this.eqDeletesByPartition = eqDeletesByPartition; this.posDeletesByPartition = posDeletesByPartition; this.posDeletesByPath = posDeletesByPath; + this.dvByPath = dvByPath; this.hasEqDeletes = globalDeletes != null || eqDeletesByPartition != null; - this.hasPosDeletes = posDeletesByPartition != null || posDeletesByPath != null; + this.hasPosDeletes = + posDeletesByPartition != null || posDeletesByPath != null || dvByPath != null; this.isEmpty = !hasEqDeletes && !hasPosDeletes; } @@ -125,6 +130,10 @@ public Iterable referencedDeleteFiles() { } } + if (dvByPath != null) { + deleteFiles = Iterables.concat(deleteFiles, dvByPath.values()); + } + return deleteFiles; } @@ -143,9 +152,16 @@ DeleteFile[] forDataFile(long sequenceNumber, DataFile file) { DeleteFile[] global = findGlobalDeletes(sequenceNumber, file); DeleteFile[] eqPartition = findEqPartitionDeletes(sequenceNumber, file); - DeleteFile[] posPartition = findPosPartitionDeletes(sequenceNumber, file); - DeleteFile[] posPath = findPathDeletes(sequenceNumber, file); - return concat(global, eqPartition, posPartition, posPath); + DeleteFile dv = findDV(sequenceNumber, file); + if (dv != null && global == null && eqPartition == null) { + return new DeleteFile[] {dv}; + } else if (dv != null) { + return concat(global, eqPartition, new DeleteFile[] {dv}); + } else { + DeleteFile[] posPartition = findPosPartitionDeletes(sequenceNumber, file); + DeleteFile[] posPath = findPathDeletes(sequenceNumber, file); + return concat(global, eqPartition, posPartition, posPath); + } } private DeleteFile[] findGlobalDeletes(long seq, DataFile dataFile) { @@ -180,6 +196,22 @@ private DeleteFile[] findPathDeletes(long seq, DataFile dataFile) { return deletes == null ? EMPTY_DELETES : deletes.filter(seq); } + private DeleteFile findDV(long seq, DataFile dataFile) { + if (dvByPath == null) { + return null; + } + + DeleteFile dv = dvByPath.get(dataFile.location()); + if (dv != null) { + ValidationException.check( + dv.dataSequenceNumber() >= seq, + "DV data sequence number (%s) must be greater than or equal to data file sequence number (%s)", + dv.dataSequenceNumber(), + seq); + } + return dv; + } + @SuppressWarnings("checkstyle:CyclomaticComplexity") private static boolean canContainEqDeletesForFile( DataFile dataFile, EqualityDeleteFile deleteFile) { @@ -434,11 +466,16 @@ DeleteFileIndex build() { PartitionMap eqDeletesByPartition = PartitionMap.create(specsById); PartitionMap posDeletesByPartition = PartitionMap.create(specsById); Map posDeletesByPath = Maps.newHashMap(); + Map dvByPath = Maps.newHashMap(); for (DeleteFile file : files) { switch (file.content()) { case POSITION_DELETES: - add(posDeletesByPath, posDeletesByPartition, file); + if (ContentFileUtil.isDV(file)) { + add(dvByPath, file); + } else { + add(posDeletesByPath, posDeletesByPartition, file); + } break; case EQUALITY_DELETES: add(globalDeletes, eqDeletesByPartition, file); @@ -453,7 +490,18 @@ DeleteFileIndex build() { globalDeletes.isEmpty() ? null : globalDeletes, eqDeletesByPartition.isEmpty() ? null : eqDeletesByPartition, posDeletesByPartition.isEmpty() ? null : posDeletesByPartition, - posDeletesByPath.isEmpty() ? null : posDeletesByPath); + posDeletesByPath.isEmpty() ? null : posDeletesByPath, + dvByPath.isEmpty() ? null : dvByPath); + } + + private void add(Map dvByPath, DeleteFile dv) { + String path = dv.referencedDataFile(); + DeleteFile existingDV = dvByPath.putIfAbsent(path, dv); + if (existingDV != null) { + throw new ValidationException( + "Can't index multiple DVs for %s: %s and %s", + path, ContentFileUtil.dvDesc(dv), ContentFileUtil.dvDesc(existingDV)); + } } private void add( diff --git a/core/src/main/java/org/apache/iceberg/FastAppend.java b/core/src/main/java/org/apache/iceberg/FastAppend.java index 1b6e1b3b52bc..51c0d5926fdb 100644 --- a/core/src/main/java/org/apache/iceberg/FastAppend.java +++ b/core/src/main/java/org/apache/iceberg/FastAppend.java @@ -24,7 +24,6 @@ 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; @@ -32,15 +31,9 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; 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 DataFileSet newFiles = DataFileSet.create(); @@ -52,8 +45,7 @@ class FastAppend extends SnapshotProducer implements AppendFiles { FastAppend(String tableName, TableOperations ops) { super(ops); this.tableName = tableName; - this.ops = ops; - this.spec = ops.current().spec(); + this.spec = ops().current().spec(); } @Override @@ -75,7 +67,8 @@ protected String operation() { @Override protected Map summary() { summaryBuilder.setPartitionSummaryLimit( - ops.current() + ops() + .current() .propertyAsInt( TableProperties.WRITE_PARTITION_SUMMARY_LIMIT, TableProperties.WRITE_PARTITION_SUMMARY_LIMIT_DEFAULT)); @@ -124,8 +117,8 @@ public FastAppend appendManifest(ManifestFile manifest) { } private ManifestFile copyManifest(ManifestFile manifest) { - TableMetadata current = ops.current(); - InputFile toCopy = ops.io().newInputFile(manifest); + TableMetadata current = ops().current(); + InputFile toCopy = ops().io().newInputFile(manifest); EncryptedOutputFile newManifestFile = newManifestOutputFile(); return ManifestFiles.copyAppendManifest( current.formatVersion(), @@ -157,7 +150,7 @@ public List apply(TableMetadata base, Snapshot snapshot) { Iterables.addAll(manifests, appendManifestsWithMetadata); if (snapshot != null) { - manifests.addAll(snapshot.allManifests(ops.io())); + manifests.addAll(snapshot.allManifests(ops().io())); } return manifests; @@ -166,7 +159,7 @@ public List apply(TableMetadata base, Snapshot snapshot) { @Override public Object updateEvent() { long snapshotId = snapshotId(); - Snapshot snapshot = ops.current().snapshot(snapshotId); + Snapshot snapshot = ops().current().snapshot(snapshotId); long sequenceNumber = snapshot.sequenceNumber(); return new CreateSnapshotEvent( tableName, operation(), snapshotId, sequenceNumber, snapshot.summary()); diff --git a/core/src/main/java/org/apache/iceberg/FileMetadata.java b/core/src/main/java/org/apache/iceberg/FileMetadata.java index 9a201d1b3b6f..15936ec8760c 100644 --- a/core/src/main/java/org/apache/iceberg/FileMetadata.java +++ b/core/src/main/java/org/apache/iceberg/FileMetadata.java @@ -59,6 +59,9 @@ public static class Builder { private ByteBuffer keyMetadata = null; private Integer sortOrderId = null; private List splitOffsets = null; + private String referencedDataFile = null; + private Long contentOffset = null; + private Long contentSizeInBytes = null; Builder(PartitionSpec spec) { this.spec = spec; @@ -91,7 +94,7 @@ public Builder copy(DeleteFile toCopy) { this.partitionData = DataFiles.copyPartitionData(spec, toCopy.partition(), partitionData); } this.content = toCopy.content(); - this.filePath = toCopy.path().toString(); + this.filePath = toCopy.location(); this.format = toCopy.format(); this.recordCount = toCopy.recordCount(); this.fileSizeInBytes = toCopy.fileSizeInBytes(); @@ -220,6 +223,25 @@ public Builder withSortOrder(SortOrder newSortOrder) { return this; } + public Builder withReferencedDataFile(CharSequence newReferencedDataFile) { + if (newReferencedDataFile != null) { + this.referencedDataFile = newReferencedDataFile.toString(); + } else { + this.referencedDataFile = null; + } + return this; + } + + public Builder withContentOffset(long newContentOffset) { + this.contentOffset = newContentOffset; + return this; + } + + public Builder withContentSizeInBytes(long newContentSizeInBytes) { + this.contentSizeInBytes = newContentSizeInBytes; + return this; + } + public DeleteFile build() { Preconditions.checkArgument(filePath != null, "File path is required"); if (format == null) { @@ -230,6 +252,15 @@ public DeleteFile build() { Preconditions.checkArgument(fileSizeInBytes >= 0, "File size is required"); Preconditions.checkArgument(recordCount >= 0, "Record count is required"); + if (format == FileFormat.PUFFIN) { + Preconditions.checkArgument(contentOffset != null, "Content offset is required for DV"); + Preconditions.checkArgument(contentSizeInBytes != null, "Content size is required for DV"); + } else { + Preconditions.checkArgument(contentOffset == null, "Content offset can only be set for DV"); + Preconditions.checkArgument( + contentSizeInBytes == null, "Content size can only be set for DV"); + } + switch (content) { case POSITION_DELETES: Preconditions.checkArgument( @@ -262,7 +293,10 @@ public DeleteFile build() { equalityFieldIds, sortOrderId, splitOffsets, - keyMetadata); + keyMetadata, + referencedDataFile, + contentOffset, + contentSizeInBytes); } } } diff --git a/core/src/main/java/org/apache/iceberg/GenericDataFile.java b/core/src/main/java/org/apache/iceberg/GenericDataFile.java index 7b99e7b60ab8..a61cc1e0fb72 100644 --- a/core/src/main/java/org/apache/iceberg/GenericDataFile.java +++ b/core/src/main/java/org/apache/iceberg/GenericDataFile.java @@ -64,7 +64,10 @@ class GenericDataFile extends BaseFile implements DataFile { splitOffsets, null /* no equality field IDs */, sortOrderId, - keyMetadata); + keyMetadata, + null /* no referenced data file */, + null /* no content offset */, + null /* no content size */); } /** diff --git a/core/src/main/java/org/apache/iceberg/GenericDeleteFile.java b/core/src/main/java/org/apache/iceberg/GenericDeleteFile.java index 77e0d8505af6..9205551f24b3 100644 --- a/core/src/main/java/org/apache/iceberg/GenericDeleteFile.java +++ b/core/src/main/java/org/apache/iceberg/GenericDeleteFile.java @@ -48,7 +48,10 @@ class GenericDeleteFile extends BaseFile implements DeleteFile { int[] equalityFieldIds, Integer sortOrderId, List splitOffsets, - ByteBuffer keyMetadata) { + ByteBuffer keyMetadata, + String referencedDataFile, + Long contentOffset, + Long contentSizeInBytes) { super( specId, content, @@ -66,7 +69,10 @@ class GenericDeleteFile extends BaseFile implements DeleteFile { splitOffsets, equalityFieldIds, sortOrderId, - keyMetadata); + keyMetadata, + referencedDataFile, + contentOffset, + contentSizeInBytes); } /** diff --git a/core/src/main/java/org/apache/iceberg/GenericManifestFile.java b/core/src/main/java/org/apache/iceberg/GenericManifestFile.java index 7707c57a6905..a079f5fb7bd4 100644 --- a/core/src/main/java/org/apache/iceberg/GenericManifestFile.java +++ b/core/src/main/java/org/apache/iceberg/GenericManifestFile.java @@ -87,6 +87,26 @@ public GenericManifestFile(Schema avroSchema) { this.keyMetadata = null; } + GenericManifestFile(InputFile file, int specId, long snapshotId) { + super(ManifestFile.schema().columns().size()); + this.avroSchema = AVRO_SCHEMA; + this.file = file; + this.manifestPath = file.location(); + this.length = null; // lazily loaded from file + this.specId = specId; + this.sequenceNumber = 0; + this.minSequenceNumber = 0; + this.snapshotId = snapshotId; + this.addedFilesCount = null; + this.addedRowsCount = null; + this.existingFilesCount = null; + this.existingRowsCount = null; + this.deletedFilesCount = null; + this.deletedRowsCount = null; + this.partitions = null; + this.keyMetadata = null; + } + /** Adjust the arg order to avoid conflict with the public constructor below */ GenericManifestFile( String path, @@ -167,7 +187,13 @@ private GenericManifestFile(GenericManifestFile toCopy) { super(toCopy); this.avroSchema = toCopy.avroSchema; this.manifestPath = toCopy.manifestPath; - this.length = toCopy.length; + try { + this.length = toCopy.length(); + } catch (UnsupportedOperationException e) { + // Can be removed when embedded manifests are dropped + // DummyFileIO does not support .length() + this.length = null; + } this.specId = toCopy.specId; this.content = toCopy.content; this.sequenceNumber = toCopy.sequenceNumber; diff --git a/core/src/main/java/org/apache/iceberg/IncrementalFileCleanup.java b/core/src/main/java/org/apache/iceberg/IncrementalFileCleanup.java index 60ad46e8e864..e1648514ef0e 100644 --- a/core/src/main/java/org/apache/iceberg/IncrementalFileCleanup.java +++ b/core/src/main/java/org/apache/iceberg/IncrementalFileCleanup.java @@ -293,7 +293,7 @@ private Set findFilesToDelete( if (entry.status() == ManifestEntry.Status.DELETED && !validIds.contains(entry.snapshotId())) { // use toString to ensure the path will not change (Utf8 is reused) - filesToDelete.add(entry.file().path().toString()); + filesToDelete.add(entry.file().location()); } } } catch (IOException e) { @@ -317,7 +317,7 @@ private Set findFilesToDelete( // delete any ADDED file from manifests that were reverted if (entry.status() == ManifestEntry.Status.ADDED) { // use toString to ensure the path will not change (Utf8 is reused) - filesToDelete.add(entry.file().path().toString()); + filesToDelete.add(entry.file().location()); } } } catch (IOException e) { 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/ManifestFiles.java b/core/src/main/java/org/apache/iceberg/ManifestFiles.java index f630bb3eb743..c9f6b783b93f 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestFiles.java +++ b/core/src/main/java/org/apache/iceberg/ManifestFiles.java @@ -96,7 +96,7 @@ public static void dropCache(FileIO fileIO) { public static CloseableIterable readPaths(ManifestFile manifest, FileIO io) { return CloseableIterable.transform( read(manifest, io, null).select(ImmutableList.of("file_path")).liveEntries(), - entry -> entry.file().path().toString()); + entry -> entry.file().location()); } /** diff --git a/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java b/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java index fddb1a161637..840d93ec1055 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java +++ b/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java @@ -40,6 +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.ManifestFileUtil; import org.apache.iceberg.util.Pair; @@ -68,15 +69,17 @@ public String partition() { private final Map specsById; private final PartitionSet deleteFilePartitions; + private final Set deleteFiles = newFileSet(); + private final Set manifestsWithDeletes = Sets.newHashSet(); private final PartitionSet dropPartitions; private final CharSequenceSet deletePaths = CharSequenceSet.empty(); - private final Set deleteFiles = newFileSet(); private Expression deleteExpression = Expressions.alwaysFalse(); private long minSequenceNumber = 0; private boolean failAnyDelete = false; private boolean failMissingDeletePaths = false; private int duplicateDeleteCount = 0; private boolean caseSensitive = true; + private boolean allDeletesReferenceManifests = true; // cache filtered manifests to avoid extra work when commits fail. private final Map filteredManifests = Maps.newConcurrentMap(); @@ -121,6 +124,7 @@ protected void deleteByRowFilter(Expression expr) { Preconditions.checkNotNull(expr, "Cannot delete files using filter: null"); invalidateFilteredCache(); this.deleteExpression = Expressions.or(deleteExpression, expr); + this.allDeletesReferenceManifests = false; } /** Add a partition tuple to drop from the table during the delete phase. */ @@ -128,6 +132,7 @@ protected void dropPartition(int specId, StructLike partition) { Preconditions.checkNotNull(partition, "Cannot delete files in invalid partition: null"); invalidateFilteredCache(); dropPartitions.add(specId, partition); + this.allDeletesReferenceManifests = false; } /** @@ -154,6 +159,13 @@ void caseSensitive(boolean newCaseSensitive) { void delete(F file) { Preconditions.checkNotNull(file, "Cannot delete file: null"); invalidateFilteredCache(); + + if (file.manifestLocation() == null) { + this.allDeletesReferenceManifests = false; + } else { + manifestsWithDeletes.add(file.manifestLocation()); + } + deleteFiles.add(file); deleteFilePartitions.add(file.specId(), file.partition()); } @@ -162,6 +174,7 @@ void delete(F file) { void delete(CharSequence path) { Preconditions.checkNotNull(path, "Cannot delete file path: null"); invalidateFilteredCache(); + this.allDeletesReferenceManifests = false; deletePaths.add(path); } @@ -185,6 +198,7 @@ List filterManifests(Schema tableSchema, List manife return ImmutableList.of(); } + boolean trustManifestReferences = canTrustManifestReferences(manifests); ManifestFile[] filtered = new ManifestFile[manifests.size()]; // open all of the manifest files in parallel, use index to avoid reordering Tasks.range(filtered.length) @@ -193,7 +207,8 @@ List filterManifests(Schema tableSchema, List manife .executeWith(workerPoolSupplier.get()) .run( index -> { - ManifestFile manifest = filterManifest(tableSchema, manifests.get(index)); + ManifestFile manifest = + filterManifest(tableSchema, manifests.get(index), trustManifestReferences); filtered[index] = manifest; }); @@ -202,6 +217,16 @@ List filterManifests(Schema tableSchema, List manife return Arrays.asList(filtered); } + // Use the current set of referenced manifests as a source of truth when it's a subset of all + // manifests and all removals which were performed reference manifests. + // If a manifest without live files is not in the trusted referenced set, this means that the + // manifest has no deleted entries and does not need to be rewritten. + private boolean canTrustManifestReferences(List manifests) { + Set manifestLocations = + manifests.stream().map(ManifestFile::path).collect(Collectors.toSet()); + return allDeletesReferenceManifests && manifestLocations.containsAll(manifestsWithDeletes); + } + /** * Creates a snapshot summary builder with the files deleted from the set of filtered manifests. * @@ -246,7 +271,7 @@ private void validateRequiredDeletes(ManifestFile... manifests) { CharSequenceSet deletedFilePaths = deletedFiles.stream() - .map(ContentFile::path) + .map(ContentFile::location) .collect(Collectors.toCollection(CharSequenceSet::empty)); ValidationException.check( @@ -307,14 +332,14 @@ private void invalidateFilteredCache() { /** * @return a ManifestReader that is a filtered version of the input manifest. */ - private ManifestFile filterManifest(Schema tableSchema, ManifestFile manifest) { + private ManifestFile filterManifest( + Schema tableSchema, ManifestFile manifest, boolean trustManifestReferences) { ManifestFile cached = filteredManifests.get(manifest); if (cached != null) { return cached; } - boolean hasLiveFiles = manifest.hasAddedFiles() || manifest.hasExistingFiles(); - if (!hasLiveFiles || !canContainDeletedFiles(manifest)) { + if (!canContainDeletedFiles(manifest, trustManifestReferences)) { filteredManifests.put(manifest, manifest); return manifest; } @@ -323,72 +348,80 @@ private ManifestFile filterManifest(Schema tableSchema, ManifestFile manifest) { PartitionSpec spec = reader.spec(); PartitionAndMetricsEvaluator evaluator = new PartitionAndMetricsEvaluator(tableSchema, spec, deleteExpression); - // this assumes that the manifest doesn't have files to remove and streams through the // manifest without copying data. if a manifest does have a file to remove, this will break // out of the loop and move on to filtering the manifest. - boolean hasDeletedFiles = manifestHasDeletedFiles(evaluator, reader); - if (!hasDeletedFiles) { + if (manifestHasDeletedFiles(evaluator, manifest, reader)) { + return filterManifestWithDeletedFiles(evaluator, manifest, reader); + } else { filteredManifests.put(manifest, manifest); return manifest; } - - return filterManifestWithDeletedFiles(evaluator, manifest, reader); - } catch (IOException e) { throw new RuntimeIOException(e, "Failed to close manifest: %s", manifest); } } - private boolean canContainDeletedFiles(ManifestFile manifest) { - boolean canContainExpressionDeletes; + private boolean canContainDeletedFiles(ManifestFile manifest, boolean trustManifestReferences) { + if (hasNoLiveFiles(manifest)) { + return false; + } + + if (trustManifestReferences) { + return manifestsWithDeletes.contains(manifest.path()); + } + + return canContainDroppedFiles(manifest) + || canContainExpressionDeletes(manifest) + || canContainDroppedPartitions(manifest); + } + + private boolean hasNoLiveFiles(ManifestFile manifest) { + return !manifest.hasAddedFiles() && !manifest.hasExistingFiles(); + } + + private boolean canContainExpressionDeletes(ManifestFile manifest) { if (deleteExpression != null && deleteExpression != Expressions.alwaysFalse()) { ManifestEvaluator manifestEvaluator = ManifestEvaluator.forRowFilter( deleteExpression, specsById.get(manifest.partitionSpecId()), caseSensitive); - canContainExpressionDeletes = manifestEvaluator.eval(manifest); - } else { - canContainExpressionDeletes = false; + return manifestEvaluator.eval(manifest); } - boolean canContainDroppedPartitions; + return false; + } + + private boolean canContainDroppedPartitions(ManifestFile manifest) { if (!dropPartitions.isEmpty()) { - canContainDroppedPartitions = - ManifestFileUtil.canContainAny(manifest, dropPartitions, specsById); - } else { - canContainDroppedPartitions = false; + return ManifestFileUtil.canContainAny(manifest, dropPartitions, specsById); } - boolean canContainDroppedFiles; + return false; + } + + private boolean canContainDroppedFiles(ManifestFile manifest) { if (!deletePaths.isEmpty()) { - canContainDroppedFiles = true; + return true; } 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); - } else { - canContainDroppedFiles = false; + return ManifestFileUtil.canContainAny(manifest, deleteFilePartitions, specsById); } - boolean canContainDropBySeq = - manifest.content() == ManifestContent.DELETES - && manifest.minSequenceNumber() < minSequenceNumber; - - return canContainExpressionDeletes - || canContainDroppedPartitions - || canContainDroppedFiles - || canContainDropBySeq; + return false; } @SuppressWarnings({"CollectionUndefinedEquality", "checkstyle:CyclomaticComplexity"}) private boolean manifestHasDeletedFiles( - PartitionAndMetricsEvaluator evaluator, ManifestReader reader) { + PartitionAndMetricsEvaluator evaluator, ManifestFile manifest, ManifestReader reader) { + if (manifestsWithDeletes.contains(manifest.path())) { + return true; + } + boolean isDelete = reader.isDeleteManifestReader(); for (ManifestEntry entry : reader.liveEntries()) { F file = entry.file(); boolean markedForDelete = - deletePaths.contains(file.path()) + deletePaths.contains(file.location()) || deleteFiles.contains(file) || dropPartitions.contains(file.specId(), file.partition()) || (isDelete @@ -431,50 +464,48 @@ private ManifestFile filterManifestWithDeletedFiles( ManifestWriter writer = newManifestWriter(reader.spec()); try { reader - .entries() + .liveEntries() .forEach( entry -> { F file = entry.file(); boolean markedForDelete = - deletePaths.contains(file.path()) + deletePaths.contains(file.location()) || deleteFiles.contains(file) || dropPartitions.contains(file.specId(), file.partition()) || (isDelete && entry.isLive() && entry.dataSequenceNumber() > 0 && entry.dataSequenceNumber() < minSequenceNumber); - if (entry.status() != ManifestEntry.Status.DELETED) { - if (markedForDelete || evaluator.rowsMightMatch(file)) { - boolean allRowsMatch = markedForDelete || evaluator.rowsMustMatch(file); - ValidationException.check( - allRowsMatch - || 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.location()); - - if (allRowsMatch) { - writer.delete(entry); - - if (deletedFiles.contains(file)) { - LOG.warn( - "Deleting a duplicate path from manifest {}: {}", - manifest.path(), - 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(file.copyWithoutStats()); - } + if (markedForDelete || evaluator.rowsMightMatch(file)) { + boolean allRowsMatch = markedForDelete || evaluator.rowsMustMatch(file); + ValidationException.check( + allRowsMatch + || 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.location()); + + if (allRowsMatch) { + writer.delete(entry); + + if (deletedFiles.contains(file)) { + LOG.warn( + "Deleting a duplicate path from manifest {}: {}", + manifest.path(), + file.location()); + duplicateDeleteCount += 1; } else { - writer.existing(entry); + // only add the file to deletes if it is a new delete + // this keeps the snapshot summary accurate for non-duplicate data + deletedFiles.add(file.copyWithoutStats()); } - } else { writer.existing(entry); } + + } else { + writer.existing(entry); } }); } finally { 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 2209b348227d..75dd7410115e 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -30,7 +30,6 @@ 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.ValidationException; @@ -42,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; @@ -50,11 +48,13 @@ 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.ContentFileUtil; 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; +import org.apache.iceberg.util.Tasks; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -72,9 +72,11 @@ abstract class MergingSnapshotProducer extends SnapshotProducer { // delete files can be added in "overwrite" or "delete" operations private static final Set VALIDATE_ADDED_DELETE_FILES_OPERATIONS = ImmutableSet.of(DataOperations.OVERWRITE, DataOperations.DELETE); + // DVs can be added in "overwrite", "delete", and "replace" operations + private static final Set VALIDATE_ADDED_DVS_OPERATIONS = + ImmutableSet.of(DataOperations.OVERWRITE, DataOperations.DELETE, DataOperations.REPLACE); private final String tableName; - private final TableOperations ops; private final SnapshotSummary.Builder summaryBuilder = SnapshotSummary.builder(); private final ManifestMergeManager mergeManager; private final ManifestFilterManager filterManager; @@ -82,11 +84,10 @@ abstract class MergingSnapshotProducer extends SnapshotProducer { private final ManifestFilterManager deleteFilterManager; // update data - private final Map> newDataFilesBySpec = Maps.newHashMap(); - private final DataFileSet newDataFiles = DataFileSet.create(); - private final DeleteFileSet newDeleteFiles = DeleteFileSet.create(); + private final Map newDataFilesBySpec = Maps.newHashMap(); private Long newDataFilesDataSequenceNumber; - private final Map> newDeleteFilesBySpec = Maps.newHashMap(); + private final Map newDeleteFilesBySpec = Maps.newHashMap(); + private final Set newDVRefs = Sets.newHashSet(); private final List appendManifests = Lists.newArrayList(); private final List rewrittenAppendManifests = Lists.newArrayList(); private final SnapshotSummary.Builder addedFilesSummary = SnapshotSummary.builder(); @@ -106,7 +107,6 @@ abstract class MergingSnapshotProducer extends SnapshotProducer { MergingSnapshotProducer(String tableName, TableOperations ops) { super(ops); this.tableName = tableName; - this.ops = ops; long targetSizeBytes = ops.current() .propertyAsLong(MANIFEST_TARGET_SIZE_BYTES, MANIFEST_TARGET_SIZE_BYTES_DEFAULT); @@ -142,18 +142,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() { @@ -161,12 +156,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() { @@ -236,47 +228,83 @@ protected boolean addsDeleteFiles() { /** Add a data file to the new snapshot. */ protected void add(DataFile file) { Preconditions.checkNotNull(file, "Invalid data file: null"); - if (newDataFiles.add(file)) { - PartitionSpec fileSpec = ops.current().spec(file.specId()); - Preconditions.checkArgument( - fileSpec != null, - "Cannot find partition spec %s for data file: %s", - file.specId(), - file.path()); - - addedFilesSummary.addedFile(fileSpec, file); + 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 dataFiles = - newDataFilesBySpec.computeIfAbsent(fileSpec, ignored -> Lists.newArrayList()); - dataFiles.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)); + validateNewDeleteFile(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)); + validateNewDeleteFile(file); + 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 (newDeleteFiles.add(fileHolder.deleteFile())) { - 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; + if (ContentFileUtil.isDV(file)) { + newDVRefs.add(file.referencedDataFile()); + } } } + protected void validateNewDeleteFile(DeleteFile file) { + Preconditions.checkNotNull(file, "Invalid delete file: null"); + switch (formatVersion()) { + case 1: + throw new IllegalArgumentException("Deletes are supported in V2 and above"); + case 2: + Preconditions.checkArgument( + file.content() == FileContent.EQUALITY_DELETES || !ContentFileUtil.isDV(file), + "Must not use DVs for position deletes in V2: %s", + ContentFileUtil.dvDesc(file)); + break; + case 3: + Preconditions.checkArgument( + file.content() == FileContent.EQUALITY_DELETES || ContentFileUtil.isDV(file), + "Must use DVs for position deletes in V%s: %s", + formatVersion(), + file.location()); + break; + default: + throw new IllegalArgumentException("Unsupported format version: " + formatVersion()); + } + } + + private int formatVersion() { + return ops().current().formatVersion(); + } + /** Add all files in a manifest to the new snapshot. */ protected void add(ManifestFile manifest) { Preconditions.checkArgument( @@ -292,8 +320,8 @@ protected void add(ManifestFile manifest) { } private ManifestFile copyManifest(ManifestFile manifest) { - TableMetadata current = ops.current(); - InputFile toCopy = ops.io().newInputFile(manifest); + TableMetadata current = ops().current(); + InputFile toCopy = ops().io().newInputFile(manifest); EncryptedOutputFile newManifestFile = newManifestOutputFile(); return ManifestFiles.copyAppendManifest( current.formatVersion(), @@ -325,7 +353,7 @@ protected void validateAddedDataFiles( "Found conflicting files that can contain records matching partitions %s: %s", partitionSet, Iterators.toString( - Iterators.transform(conflicts, entry -> entry.file().path().toString()))); + Iterators.transform(conflicts, entry -> entry.file().location().toString()))); } } catch (IOException e) { @@ -356,7 +384,7 @@ protected void validateAddedDataFiles( "Found conflicting files that can contain records matching %s: %s", conflictDetectionFilter, Iterators.toString( - Iterators.transform(conflicts, entry -> entry.file().path().toString()))); + Iterators.transform(conflicts, entry -> entry.file().location().toString()))); } } catch (IOException e) { @@ -397,7 +425,7 @@ private CloseableIterable> addedDataFiles( Set newSnapshots = history.second(); ManifestGroup manifestGroup = - new ManifestGroup(ops.io(), manifests, ImmutableList.of()) + new ManifestGroup(ops().io(), manifests, ImmutableList.of()) .caseSensitive(caseSensitive) .filterManifestEntries(entry -> newSnapshots.contains(entry.snapshotId())) .specsById(base.specsById()) @@ -520,7 +548,7 @@ protected void validateNoNewDeleteFiles( deletes.isEmpty(), "Found new conflicting delete files that can apply to records matching %s: %s", dataFilter, - Iterables.transform(deletes.referencedDeleteFiles(), ContentFile::path)); + Iterables.transform(deletes.referencedDeleteFiles(), ContentFile::location)); } /** @@ -540,7 +568,7 @@ protected void validateNoNewDeleteFiles( deletes.isEmpty(), "Found new conflicting delete files that can apply to records matching %s: %s", partitionSet, - Iterables.transform(deletes.referencedDeleteFiles(), ContentFile::path)); + Iterables.transform(deletes.referencedDeleteFiles(), ContentFile::location)); } /** @@ -560,7 +588,7 @@ protected DeleteFileIndex addedDeleteFiles( Snapshot parent) { // if there is no current table state, return empty delete file index if (parent == null || base.formatVersion() < 2) { - return DeleteFileIndex.builderFor(ops.io(), ImmutableList.of()) + return DeleteFileIndex.builderFor(ops().io(), ImmutableList.of()) .specsById(base.specsById()) .build(); } @@ -598,7 +626,7 @@ protected void validateDeletedDataFiles( "Found conflicting deleted files that can contain records matching %s: %s", dataFilter, Iterators.toString( - Iterators.transform(conflicts, entry -> entry.file().path().toString()))); + Iterators.transform(conflicts, entry -> entry.file().location().toString()))); } } catch (IOException e) { @@ -627,7 +655,7 @@ protected void validateDeletedDataFiles( "Found conflicting deleted files that can apply to records matching %s: %s", partitionSet, Iterators.toString( - Iterators.transform(conflicts, entry -> entry.file().path().toString()))); + Iterators.transform(conflicts, entry -> entry.file().location().toString()))); } } catch (IOException e) { @@ -668,7 +696,7 @@ private CloseableIterable> deletedDataFiles( Set newSnapshots = history.second(); ManifestGroup manifestGroup = - new ManifestGroup(ops.io(), manifests, ImmutableList.of()) + new ManifestGroup(ops().io(), manifests, ImmutableList.of()) .caseSensitive(caseSensitive) .filterManifestEntries(entry -> newSnapshots.contains(entry.snapshotId())) .filterManifestEntries(entry -> entry.status().equals(ManifestEntry.Status.DELETED)) @@ -707,10 +735,10 @@ private DeleteFileIndex buildDeleteFileIndex( Expression dataFilter, PartitionSet partitionSet) { DeleteFileIndex.Builder builder = - DeleteFileIndex.builderFor(ops.io(), deleteManifests) + DeleteFileIndex.builderFor(ops().io(), deleteManifests) .afterSequenceNumber(startingSequenceNumber) .caseSensitive(caseSensitive) - .specsById(ops.current().specsById()); + .specsById(ops().current().specsById()); if (dataFilter != null) { builder.filterData(dataFilter); @@ -748,12 +776,12 @@ protected void validateDataFilesExist( Set newSnapshots = history.second(); ManifestGroup matchingDeletesGroup = - new ManifestGroup(ops.io(), manifests, ImmutableList.of()) + new ManifestGroup(ops().io(), manifests, ImmutableList.of()) .filterManifestEntries( entry -> entry.status() != ManifestEntry.Status.ADDED && newSnapshots.contains(entry.snapshotId()) - && requiredDataFiles.contains(entry.file().path())) + && requiredDataFiles.contains(entry.file().location())) .specsById(base.specsById()) .ignoreExisting(); @@ -767,7 +795,7 @@ protected void validateDataFilesExist( throw new ValidationException( "Cannot commit, missing data files: %s", Iterators.toString( - Iterators.transform(deletes, entry -> entry.file().path().toString()))); + Iterators.transform(deletes, entry -> entry.file().location().toString()))); } } catch (IOException e) { @@ -775,6 +803,58 @@ protected void validateDataFilesExist( } } + // validates there are no concurrently added DVs for referenced data files + protected void validateAddedDVs( + TableMetadata base, + Long startingSnapshotId, + Expression conflictDetectionFilter, + Snapshot parent) { + // skip if there is no current table state or this operation doesn't add new DVs + if (parent == null || newDVRefs.isEmpty()) { + return; + } + + Pair, Set> history = + validationHistory( + base, + startingSnapshotId, + VALIDATE_ADDED_DVS_OPERATIONS, + ManifestContent.DELETES, + parent); + List newDeleteManifests = history.first(); + Set newSnapshotIds = history.second(); + + Tasks.foreach(newDeleteManifests) + .stopOnFailure() + .throwFailureWhenFinished() + .executeWith(workerPool()) + .run(manifest -> validateAddedDVs(manifest, conflictDetectionFilter, newSnapshotIds)); + } + + private void validateAddedDVs( + ManifestFile manifest, Expression conflictDetectionFilter, Set newSnapshotIds) { + try (CloseableIterable> entries = + ManifestFiles.readDeleteManifest(manifest, ops().io(), ops().current().specsById()) + .filterRows(conflictDetectionFilter) + .caseSensitive(caseSensitive) + .liveEntries()) { + + for (ManifestEntry entry : entries) { + DeleteFile file = entry.file(); + if (newSnapshotIds.contains(entry.snapshotId()) && ContentFileUtil.isDV(file)) { + ValidationException.check( + !newDVRefs.contains(file.referencedDataFile()), + "Found concurrently added DV for %s: %s", + file.referencedDataFile(), + ContentFileUtil.dvDesc(file)); + } + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + // returns newly added manifests and snapshot IDs between the starting and parent snapshots private Pair, Set> validationHistory( TableMetadata base, Long startingSnapshotId, @@ -793,13 +873,13 @@ private Pair, Set> validationHistory( if (matchingOperations.contains(currentSnapshot.operation())) { newSnapshots.add(currentSnapshot.snapshotId()); if (content == ManifestContent.DATA) { - for (ManifestFile manifest : currentSnapshot.dataManifests(ops.io())) { + for (ManifestFile manifest : currentSnapshot.dataManifests(ops().io())) { if (manifest.snapshotId() == currentSnapshot.snapshotId()) { manifests.add(manifest); } } } else { - for (ManifestFile manifest : currentSnapshot.deleteManifests(ops.io())) { + for (ManifestFile manifest : currentSnapshot.deleteManifests(ops().io())) { if (manifest.snapshotId() == currentSnapshot.snapshotId()) { manifests.add(manifest); } @@ -820,7 +900,8 @@ private Pair, Set> validationHistory( @Override protected Map summary() { summaryBuilder.setPartitionSummaryLimit( - ops.current() + ops() + .current() .propertyAsInt( TableProperties.WRITE_PARTITION_SUMMARY_LIMIT, TableProperties.WRITE_PARTITION_SUMMARY_LIMIT_DEFAULT)); @@ -833,7 +914,7 @@ public List apply(TableMetadata base, Snapshot snapshot) { List filtered = filterManager.filterManifests( SnapshotUtil.schemaFor(base, targetBranch()), - snapshot != null ? snapshot.dataManifests(ops.io()) : null); + snapshot != null ? snapshot.dataManifests(ops().io()) : null); long minDataSequenceNumber = filtered.stream() .map(ManifestFile::minSequenceNumber) @@ -847,7 +928,7 @@ public List apply(TableMetadata base, Snapshot snapshot) { List filteredDeletes = deleteFilterManager.filterManifests( SnapshotUtil.schemaFor(base, targetBranch()), - snapshot != null ? snapshot.deleteManifests(ops.io()) : null); + snapshot != null ? snapshot.deleteManifests(ops().io()) : null); // only keep manifests that have live data files or that were written by this commit Predicate shouldKeep = @@ -877,7 +958,7 @@ public List apply(TableMetadata base, Snapshot snapshot) { @Override public Object updateEvent() { long snapshotId = snapshotId(); - Snapshot justSaved = ops.refresh().snapshot(snapshotId); + Snapshot justSaved = ops().refresh().snapshot(snapshotId); long sequenceNumber = TableMetadata.INVALID_SEQUENCE_NUMBER; Map summary; if (justSaved == null) { @@ -972,9 +1053,9 @@ private List newDataFilesAsManifests() { if (cachedNewDataManifests.isEmpty()) { newDataFilesBySpec.forEach( - (dataSpec, dataFiles) -> { + (specId, dataFiles) -> { List newDataManifests = - writeDataManifests(dataFiles, newDataFilesDataSequenceNumber, dataSpec); + writeDataManifests(dataFiles, newDataFilesDataSequenceNumber, spec(specId)); cachedNewDataManifests.addAll(newDataManifests); }); this.hasNewDataFiles = false; @@ -1004,7 +1085,7 @@ private List newDeleteFilesAsManifests() { if (cachedNewDeleteManifests.isEmpty()) { newDeleteFilesBySpec.forEach( (specId, deleteFiles) -> { - PartitionSpec spec = ops.current().spec(specId); + PartitionSpec spec = ops().current().spec(specId); List newDeleteManifests = writeDeleteManifests(deleteFiles, spec); cachedNewDeleteManifests.addAll(newDeleteManifests); }); @@ -1017,7 +1098,7 @@ private List newDeleteFilesAsManifests() { private class DataFileFilterManager extends ManifestFilterManager { private DataFileFilterManager() { - super(ops.current().specsById(), MergingSnapshotProducer.this::workerPool); + super(ops().current().specsById(), MergingSnapshotProducer.this::workerPool); } @Override @@ -1054,7 +1135,7 @@ protected long snapshotId() { @Override protected PartitionSpec spec(int specId) { - return ops.current().spec(specId); + return ops().current().spec(specId); } @Override @@ -1075,7 +1156,7 @@ protected ManifestReader newManifestReader(ManifestFile manifest) { private class DeleteFileFilterManager extends ManifestFilterManager { private DeleteFileFilterManager() { - super(ops.current().specsById(), MergingSnapshotProducer.this::workerPool); + super(ops().current().specsById(), MergingSnapshotProducer.this::workerPool); } @Override @@ -1112,7 +1193,7 @@ protected long snapshotId() { @Override protected PartitionSpec spec(int specId) { - return ops.current().spec(specId); + return ops().current().spec(specId); } @Override diff --git a/core/src/main/java/org/apache/iceberg/MetadataUpdate.java b/core/src/main/java/org/apache/iceberg/MetadataUpdate.java index 49fb1fe01c44..ba038c196e43 100644 --- a/core/src/main/java/org/apache/iceberg/MetadataUpdate.java +++ b/core/src/main/java/org/apache/iceberg/MetadataUpdate.java @@ -86,6 +86,16 @@ class AddSchema implements MetadataUpdate { private final Schema schema; private final int lastColumnId; + public AddSchema(Schema schema) { + this(schema, schema.highestFieldId()); + } + + /** + * Set the schema + * + * @deprecated since 1.8.0, will be removed 1.9.0 or 2.0.0, use AddSchema(schema). + */ + @Deprecated public AddSchema(Schema schema, int lastColumnId) { this.schema = schema; this.lastColumnId = lastColumnId; diff --git a/core/src/main/java/org/apache/iceberg/PropertiesUpdate.java b/core/src/main/java/org/apache/iceberg/PropertiesUpdate.java index 35338a689205..9389aec50c0a 100644 --- a/core/src/main/java/org/apache/iceberg/PropertiesUpdate.java +++ b/core/src/main/java/org/apache/iceberg/PropertiesUpdate.java @@ -98,12 +98,13 @@ public Map apply() { @Override public void commit() { + // If existing table commit properties in base are corrupted, allow rectification Tasks.foreach(ops) - .retry(base.propertyAsInt(COMMIT_NUM_RETRIES, COMMIT_NUM_RETRIES_DEFAULT)) + .retry(base.propertyTryAsInt(COMMIT_NUM_RETRIES, COMMIT_NUM_RETRIES_DEFAULT)) .exponentialBackoff( - base.propertyAsInt(COMMIT_MIN_RETRY_WAIT_MS, COMMIT_MIN_RETRY_WAIT_MS_DEFAULT), - base.propertyAsInt(COMMIT_MAX_RETRY_WAIT_MS, COMMIT_MAX_RETRY_WAIT_MS_DEFAULT), - base.propertyAsInt(COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT), + base.propertyTryAsInt(COMMIT_MIN_RETRY_WAIT_MS, COMMIT_MIN_RETRY_WAIT_MS_DEFAULT), + base.propertyTryAsInt(COMMIT_MAX_RETRY_WAIT_MS, COMMIT_MAX_RETRY_WAIT_MS_DEFAULT), + base.propertyTryAsInt(COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT), 2.0 /* exponential */) .onlyRetryOn(CommitFailedException.class) .run( diff --git a/core/src/main/java/org/apache/iceberg/ScanSummary.java b/core/src/main/java/org/apache/iceberg/ScanSummary.java index 1ea171c5b2c3..5f8e66c0b450 100644 --- a/core/src/main/java/org/apache/iceberg/ScanSummary.java +++ b/core/src/main/java/org/apache/iceberg/ScanSummary.java @@ -47,6 +47,7 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.util.DateTimeUtil; import org.apache.iceberg.util.Pair; +import org.apache.iceberg.util.ScanTaskUtil; public class ScanSummary { private ScanSummary() {} @@ -294,7 +295,7 @@ PartitionMetrics updateFromCounts( private PartitionMetrics updateFromFile(ContentFile file, Long timestampMillis) { this.fileCount += 1; this.recordCount += file.recordCount(); - this.totalSize += file.fileSizeInBytes(); + this.totalSize += ScanTaskUtil.contentSizeInBytes(file); if (timestampMillis != null && (dataTimestampMillis == null || dataTimestampMillis < timestampMillis)) { this.dataTimestampMillis = timestampMillis; diff --git a/core/src/main/java/org/apache/iceberg/SchemaParser.java b/core/src/main/java/org/apache/iceberg/SchemaParser.java index a4333af1be19..27e6ed048712 100644 --- a/core/src/main/java/org/apache/iceberg/SchemaParser.java +++ b/core/src/main/java/org/apache/iceberg/SchemaParser.java @@ -49,6 +49,8 @@ private SchemaParser() {} private static final String DOC = "doc"; private static final String NAME = "name"; private static final String ID = "id"; + private static final String INITIAL_DEFAULT = "initial-default"; + private static final String WRITE_DEFAULT = "write-default"; private static final String ELEMENT_ID = "element-id"; private static final String KEY_ID = "key-id"; private static final String VALUE_ID = "value-id"; @@ -88,6 +90,17 @@ private static void toJson( if (field.doc() != null) { generator.writeStringField(DOC, field.doc()); } + + if (field.initialDefault() != null) { + generator.writeFieldName(INITIAL_DEFAULT); + SingleValueParser.toJson(field.type(), field.initialDefault(), generator); + } + + if (field.writeDefault() != null) { + generator.writeFieldName(WRITE_DEFAULT); + SingleValueParser.toJson(field.type(), field.writeDefault(), generator); + } + generator.writeEndObject(); } generator.writeEndArray(); @@ -184,6 +197,22 @@ private static Type typeFromJson(JsonNode json) { throw new IllegalArgumentException("Cannot parse type from json: " + json); } + private static Object defaultFromJson(String defaultField, Type type, JsonNode json) { + if (json.has(defaultField)) { + return SingleValueParser.fromJson(type, json.get(defaultField)); + } + + return null; + } + + private static Types.NestedField.Builder fieldBuilder(boolean isRequired, String name) { + if (isRequired) { + return Types.NestedField.required(name); + } else { + return Types.NestedField.optional(name); + } + } + private static Types.StructType structFromJson(JsonNode json) { JsonNode fieldArray = JsonUtil.get(FIELDS, json); Preconditions.checkArgument( @@ -200,13 +229,19 @@ private static Types.StructType structFromJson(JsonNode json) { String name = JsonUtil.getString(NAME, field); Type type = typeFromJson(JsonUtil.get(TYPE, field)); + Object initialDefault = defaultFromJson(INITIAL_DEFAULT, type, field); + Object writeDefault = defaultFromJson(WRITE_DEFAULT, type, field); + String doc = JsonUtil.getStringOrNull(DOC, field); boolean isRequired = JsonUtil.getBool(REQUIRED, field); - if (isRequired) { - fields.add(Types.NestedField.required(id, name, type, doc)); - } else { - fields.add(Types.NestedField.optional(id, name, type, doc)); - } + fields.add( + fieldBuilder(isRequired, name) + .withId(id) + .ofType(type) + .withDoc(doc) + .withInitialDefault(initialDefault) + .withWriteDefault(writeDefault) + .build()); } return Types.StructType.of(fields); diff --git a/core/src/main/java/org/apache/iceberg/SchemaUpdate.java b/core/src/main/java/org/apache/iceberg/SchemaUpdate.java index 069097778606..2b541080ac72 100644 --- a/core/src/main/java/org/apache/iceberg/SchemaUpdate.java +++ b/core/src/main/java/org/apache/iceberg/SchemaUpdate.java @@ -444,7 +444,7 @@ public Schema apply() { @Override public void commit() { - TableMetadata update = applyChangesToMetadata(base.updateSchema(apply(), lastColumnId)); + TableMetadata update = applyChangesToMetadata(base.updateSchema(apply())); ops.commit(base, update); } diff --git a/core/src/main/java/org/apache/iceberg/SerializableTable.java b/core/src/main/java/org/apache/iceberg/SerializableTable.java index 082e50b840dc..b7e2f6da09a4 100644 --- a/core/src/main/java/org/apache/iceberg/SerializableTable.java +++ b/core/src/main/java/org/apache/iceberg/SerializableTable.java @@ -50,6 +50,7 @@ * storage. */ public class SerializableTable implements Table, HasTableOperations, Serializable { + private static final int UNKNOWN_FORMAT_VERSION = -1; private final String name; private final String location; @@ -62,13 +63,14 @@ public class SerializableTable implements Table, HasTableOperations, Serializabl private final FileIO io; private final EncryptionManager encryption; private final Map refs; + private final UUID uuid; + private final int formatVersion; private transient volatile LocationProvider lazyLocationProvider = null; private transient volatile Table lazyTable = null; private transient volatile Schema lazySchema = null; private transient volatile Map lazySpecs = null; private transient volatile SortOrder lazySortOrder = null; - private final UUID uuid; protected SerializableTable(Table table) { this.name = table.name(); @@ -85,6 +87,7 @@ protected SerializableTable(Table table) { this.encryption = table.encryption(); this.refs = SerializableMap.copyOf(table.refs()); this.uuid = table.uuid(); + this.formatVersion = formatVersion(table); } /** @@ -158,6 +161,23 @@ public Map properties() { return properties; } + public int formatVersion() { + if (formatVersion == UNKNOWN_FORMAT_VERSION) { + throw new UnsupportedOperationException( + this.getClass().getName() + " does not have a format version"); + } + return formatVersion; + } + + private int formatVersion(Table table) { + if (table instanceof HasTableOperations) { + HasTableOperations ops = (HasTableOperations) table; + return ops.operations().current().formatVersion(); + } else { + return UNKNOWN_FORMAT_VERSION; + } + } + @Override public Schema schema() { if (lazySchema == null) { @@ -422,6 +442,12 @@ protected Table newTable(TableOperations ops, String tableName) { return MetadataTableUtils.createMetadataTableInstance(ops, baseTableName, tableName, type); } + @Override + public StaticTableOperations operations() { + throw new UnsupportedOperationException( + this.getClass().getName() + " does not support operations()"); + } + public MetadataTableType type() { return type; } diff --git a/core/src/main/java/org/apache/iceberg/SnapshotParser.java b/core/src/main/java/org/apache/iceberg/SnapshotParser.java index bc5ef6094695..b5ac3ec718ac 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotParser.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotParser.java @@ -31,8 +31,11 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.util.JsonUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class SnapshotParser { + private static final Logger LOG = LoggerFactory.getLogger(SnapshotParser.class); private SnapshotParser() {} @@ -129,17 +132,28 @@ static Snapshot fromJson(JsonNode node) { "Cannot parse summary from non-object value: %s", sNode); - ImmutableMap.Builder builder = ImmutableMap.builder(); - Iterator fields = sNode.fieldNames(); - while (fields.hasNext()) { - String field = fields.next(); - if (field.equals(OPERATION)) { - operation = JsonUtil.getString(OPERATION, sNode); - } else { - builder.put(field, JsonUtil.getString(field, sNode)); + if (sNode.size() > 0) { + ImmutableMap.Builder builder = ImmutableMap.builder(); + Iterator fields = sNode.fieldNames(); + while (fields.hasNext()) { + String field = fields.next(); + if (field.equals(OPERATION)) { + operation = JsonUtil.getString(OPERATION, sNode); + } else { + builder.put(field, JsonUtil.getString(field, sNode)); + } + } + summary = builder.build(); + + // When the operation is not found, default to overwrite + // to ensure that we can read the summary without raising an exception + if (operation == null) { + LOG.warn( + "Encountered invalid summary for snapshot {}: the field 'operation' is required but missing, setting 'operation' to overwrite", + snapshotId); + operation = DataOperations.OVERWRITE; } } - summary = builder.build(); } Integer schemaId = JsonUtil.getIntOrNull(SCHEMA_ID, node); diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 27724f787dd2..bc65e90eaeae 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -35,6 +35,7 @@ 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; @@ -79,6 +80,12 @@ 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); @@ -152,6 +159,10 @@ public ThisT scanManifestsWith(ExecutorService executorService) { return self(); } + protected TableOperations ops() { + return ops; + } + protected CommitMetrics commitMetrics() { if (commitMetrics == null) { this.commitMetrics = CommitMetrics.of(new DefaultMetricsContext()); @@ -595,20 +606,22 @@ private List writeDataFileGroup( } protected List writeDeleteManifests( - Collection files, PartitionSpec spec) { + Collection files, PartitionSpec spec) { return writeManifests(files, group -> writeDeleteFileGroup(group, spec)); } private List writeDeleteFileGroup( - Collection files, PartitionSpec spec) { + Collection files, PartitionSpec spec) { RollingManifestWriter writer = newRollingDeleteManifestWriter(spec); try (RollingManifestWriter closableWriter = writer) { - for (DeleteFileHolder file : files) { + for (DeleteFile file : files) { + Preconditions.checkArgument( + file instanceof PendingDeleteFile, "Invalid delete file: must be PendingDeleteFile"); if (file.dataSequenceNumber() != null) { - closableWriter.add(file.deleteFile(), file.dataSequenceNumber()); + closableWriter.add(file, file.dataSequenceNumber()); } else { - closableWriter.add(file.deleteFile()); + closableWriter.add(file); } } } catch (IOException e) { @@ -752,7 +765,7 @@ private static void updateTotal( } } - protected static class DeleteFileHolder { + protected static class PendingDeleteFile implements DeleteFile { private final DeleteFile deleteFile; private final Long dataSequenceNumber; @@ -762,7 +775,7 @@ protected static class DeleteFileHolder { * @param deleteFile delete file * @param dataSequenceNumber data sequence number to apply */ - DeleteFileHolder(DeleteFile deleteFile, long dataSequenceNumber) { + PendingDeleteFile(DeleteFile deleteFile, long dataSequenceNumber) { this.deleteFile = deleteFile; this.dataSequenceNumber = dataSequenceNumber; } @@ -772,17 +785,162 @@ protected static class DeleteFileHolder { * * @param deleteFile delete file */ - DeleteFileHolder(DeleteFile deleteFile) { + PendingDeleteFile(DeleteFile deleteFile) { this.deleteFile = deleteFile; this.dataSequenceNumber = null; } - public DeleteFile deleteFile() { - return deleteFile; + 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.location(); + } + + @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(); + } + + @Override + public String referencedDataFile() { + return deleteFile.referencedDataFile(); + } + + @Override + public Long contentOffset() { + return deleteFile.contentOffset(); + } + + @Override + public Long contentSizeInBytes() { + return deleteFile.contentSizeInBytes(); + } } } diff --git a/core/src/main/java/org/apache/iceberg/SnapshotSummary.java b/core/src/main/java/org/apache/iceberg/SnapshotSummary.java index 22c9df2a8eaf..6043424cd7fc 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotSummary.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotSummary.java @@ -25,6 +25,8 @@ import org.apache.iceberg.relocated.com.google.common.base.Strings; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.ContentFileUtil; +import org.apache.iceberg.util.ScanTaskUtil; public class SnapshotSummary { public static final String ADDED_FILES_PROP = "added-data-files"; @@ -35,6 +37,8 @@ public class SnapshotSummary { public static final String REMOVED_EQ_DELETE_FILES_PROP = "removed-equality-delete-files"; public static final String ADD_POS_DELETE_FILES_PROP = "added-position-delete-files"; public static final String REMOVED_POS_DELETE_FILES_PROP = "removed-position-delete-files"; + public static final String ADDED_DVS_PROP = "added-dvs"; + public static final String REMOVED_DVS_PROP = "removed-dvs"; public static final String REMOVED_DELETE_FILES_PROP = "removed-delete-files"; public static final String TOTAL_DELETE_FILES_PROP = "total-delete-files"; public static final String ADDED_RECORDS_PROP = "added-records"; @@ -221,6 +225,8 @@ private static class UpdateMetrics { private int removedEqDeleteFiles = 0; private int addedPosDeleteFiles = 0; private int removedPosDeleteFiles = 0; + private int addedDVs = 0; + private int removedDVs = 0; private int addedDeleteFiles = 0; private int removedDeleteFiles = 0; private long addedRecords = 0L; @@ -242,6 +248,8 @@ void clear() { this.removedPosDeleteFiles = 0; this.addedDeleteFiles = 0; this.removedDeleteFiles = 0; + this.addedDVs = 0; + this.removedDVs = 0; this.addedRecords = 0L; this.deletedRecords = 0L; this.addedPosDeletes = 0L; @@ -261,6 +269,8 @@ void addTo(ImmutableMap.Builder builder) { removedPosDeleteFiles > 0, builder, REMOVED_POS_DELETE_FILES_PROP, removedPosDeleteFiles); setIf(addedDeleteFiles > 0, builder, ADDED_DELETE_FILES_PROP, addedDeleteFiles); setIf(removedDeleteFiles > 0, builder, REMOVED_DELETE_FILES_PROP, removedDeleteFiles); + setIf(addedDVs > 0, builder, ADDED_DVS_PROP, addedDVs); + setIf(removedDVs > 0, builder, REMOVED_DVS_PROP, removedDVs); setIf(addedRecords > 0, builder, ADDED_RECORDS_PROP, addedRecords); setIf(deletedRecords > 0, builder, DELETED_RECORDS_PROP, deletedRecords); @@ -275,15 +285,20 @@ void addTo(ImmutableMap.Builder builder) { } void addedFile(ContentFile file) { - this.addedSize += file.fileSizeInBytes(); + this.addedSize += ScanTaskUtil.contentSizeInBytes(file); switch (file.content()) { case DATA: this.addedFiles += 1; this.addedRecords += file.recordCount(); break; case POSITION_DELETES: + DeleteFile deleteFile = (DeleteFile) file; + if (ContentFileUtil.isDV(deleteFile)) { + this.addedDVs += 1; + } else { + this.addedPosDeleteFiles += 1; + } this.addedDeleteFiles += 1; - this.addedPosDeleteFiles += 1; this.addedPosDeletes += file.recordCount(); break; case EQUALITY_DELETES: @@ -298,15 +313,20 @@ void addedFile(ContentFile file) { } void removedFile(ContentFile file) { - this.removedSize += file.fileSizeInBytes(); + this.removedSize += ScanTaskUtil.contentSizeInBytes(file); switch (file.content()) { case DATA: this.removedFiles += 1; this.deletedRecords += file.recordCount(); break; case POSITION_DELETES: + DeleteFile deleteFile = (DeleteFile) file; + if (ContentFileUtil.isDV(deleteFile)) { + this.removedDVs += 1; + } else { + this.removedPosDeleteFiles += 1; + } this.removedDeleteFiles += 1; - this.removedPosDeleteFiles += 1; this.removedPosDeletes += file.recordCount(); break; case EQUALITY_DELETES: @@ -343,6 +363,8 @@ void merge(UpdateMetrics other) { this.removedEqDeleteFiles += other.removedEqDeleteFiles; this.addedPosDeleteFiles += other.addedPosDeleteFiles; this.removedPosDeleteFiles += other.removedPosDeleteFiles; + this.addedDVs += other.addedDVs; + this.removedDVs += other.removedDVs; this.addedDeleteFiles += other.addedDeleteFiles; this.removedDeleteFiles += other.removedDeleteFiles; this.addedSize += other.addedSize; diff --git a/core/src/main/java/org/apache/iceberg/SplitPositionDeletesScanTask.java b/core/src/main/java/org/apache/iceberg/SplitPositionDeletesScanTask.java index 68c7d5f9fd88..1ec3599a9b84 100644 --- a/core/src/main/java/org/apache/iceberg/SplitPositionDeletesScanTask.java +++ b/core/src/main/java/org/apache/iceberg/SplitPositionDeletesScanTask.java @@ -80,7 +80,7 @@ public SplitPositionDeletesScanTask merge(org.apache.iceberg.ScanTask other) { @Override public String toString() { return MoreObjects.toStringHelper(this) - .add("file", file().path()) + .add("file", file().location()) .add("partition_data", file().partition()) .add("offset", offset) .add("length", length) 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/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index d20dd59d2b97..19afb7af04aa 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -134,6 +134,8 @@ static TableMetadata newTableMetadata( // break existing tables. MetricsConfig.fromProperties(properties).validateReferencedColumns(schema); + PropertyUtil.validateCommitProperties(properties); + return new Builder() .setInitialFormatVersion(formatVersion) .setCurrentSchema(freshSchema, lastColumnId.get()) @@ -293,8 +295,9 @@ public String toString() { sortOrders != null && !sortOrders.isEmpty(), "Sort orders cannot be null or empty"); Preconditions.checkArgument( formatVersion <= SUPPORTED_TABLE_FORMAT_VERSION, - "Unsupported format version: v%s", - formatVersion); + "Unsupported format version: v%s (supported: v%s)", + formatVersion, + SUPPORTED_TABLE_FORMAT_VERSION); Preconditions.checkArgument( formatVersion == 1 || uuid != null, "UUID is required in format v%s", formatVersion); Preconditions.checkArgument( @@ -486,6 +489,10 @@ public int propertyAsInt(String property, int defaultValue) { return PropertyUtil.propertyAsInt(properties, property, defaultValue); } + public int propertyTryAsInt(String property, int defaultValue) { + return PropertyUtil.propertyTryAsInt(properties, property, defaultValue); + } + pub